mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 07:01:59 +00:00
Merge ad4b201892
into e0f8b8d351
This commit is contained in:
commit
c60be7ebff
@ -344,7 +344,7 @@
|
||||
* The system table `text_log` is enabled by default. This is fully compatible with previous versions, but you may notice subtly increased disk usage on the local disk (this system table takes a tiny amount of disk space). [#67428](https://github.com/ClickHouse/ClickHouse/pull/67428) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* In previous versions, `arrayWithConstant` can be slow if asked to generate very large arrays. In the new version, it is limited to 1 GB per array. This closes [#32754](https://github.com/ClickHouse/ClickHouse/issues/32754). [#67741](https://github.com/ClickHouse/ClickHouse/pull/67741) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Fix REPLACE modifier formatting (forbid omitting brackets). [#67774](https://github.com/ClickHouse/ClickHouse/pull/67774) ([Azat Khuzhin](https://github.com/azat)).
|
||||
* Backported in [#68349](https://github.com/ClickHouse/ClickHouse/issues/68349): Reimplement `Dynamic` type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into `Dynamic` column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Backported in [#68349](https://github.com/ClickHouse/ClickHouse/issues/68349): Reimplement `Dynamic` type. Now when the limit of dynamic data types is reached new types are not cast to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into `Dynamic` column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
|
||||
#### New Feature
|
||||
* Added a new `MergeTree` setting `deduplicate_merge_projection_mode` to control the projections during merges (for specific engines) and `OPTIMIZE DEDUPLICATE` query. Supported options: `throw` (throw an exception in case the projection is not fully supported for *MergeTree engine), `drop` (remove projection during merge if it can't be merged itself consistently) and `rebuild` (rebuild projection from scratch, which is a heavy operation). [#66672](https://github.com/ClickHouse/ClickHouse/pull/66672) ([jsc0218](https://github.com/jsc0218)).
|
||||
|
@ -11,9 +11,9 @@
|
||||
*
|
||||
* In contrast to std::bit_cast can cast types of different width.
|
||||
*
|
||||
* Note: for signed types of narrower size, the casted result is zero-extended
|
||||
* Note: for signed types of narrower size, the cast result is zero-extended
|
||||
* instead of sign-extended as with regular static_cast.
|
||||
* For example, -1 Int8 (represented as 0xFF) bit_casted to UInt64
|
||||
* For example, -1 Int8 (represented as 0xFF) bit_cast to UInt64
|
||||
* gives 255 (represented as 0x00000000000000FF) instead of 0xFFFFFFFFFFFFFFFF
|
||||
*/
|
||||
template <typename To, typename From>
|
||||
|
@ -337,7 +337,7 @@ struct integer<Bits, Signed>::_impl
|
||||
|
||||
/** Here we have to use strict comparison.
|
||||
* The max_int is 2^64 - 1.
|
||||
* When casted to floating point type, it will be rounded to the closest representable number,
|
||||
* When cast to a floating point type, it will be rounded to the closest representable number,
|
||||
* which is 2^64.
|
||||
* But 2^64 is not representable in uint64_t,
|
||||
* so the maximum representable number will be strictly less.
|
||||
|
@ -1316,7 +1316,6 @@ bools
|
||||
boringssl
|
||||
boundingRatio
|
||||
bozerkins
|
||||
broadcasted
|
||||
brotli
|
||||
bson
|
||||
bsoneachrow
|
||||
@ -1342,7 +1341,6 @@ cardinalities
|
||||
cardinality
|
||||
cartesian
|
||||
cassandra
|
||||
casted
|
||||
catboost
|
||||
catboostEvaluate
|
||||
categoricalInformationValue
|
||||
|
@ -56,7 +56,7 @@ sidebar_label: 2023
|
||||
|
||||
#### Improvement
|
||||
* This is the second part of Kusto Query Language dialect support. [Phase 1 implementation ](https://github.com/ClickHouse/ClickHouse/pull/37961) has been merged. [#42510](https://github.com/ClickHouse/ClickHouse/pull/42510) ([larryluogit](https://github.com/larryluogit)).
|
||||
* Op processors IDs are raw ptrs casted to UInt64. Print it in a prettier manner:. [#48852](https://github.com/ClickHouse/ClickHouse/pull/48852) ([Vlad Seliverstov](https://github.com/behebot)).
|
||||
* Op processors IDs are raw ptrs cast to UInt64. Print it in a prettier manner:. [#48852](https://github.com/ClickHouse/ClickHouse/pull/48852) ([Vlad Seliverstov](https://github.com/behebot)).
|
||||
* Creating a direct dictionary with a lifetime field set will be rejected at create time. Fixes: [#27861](https://github.com/ClickHouse/ClickHouse/issues/27861). [#49043](https://github.com/ClickHouse/ClickHouse/pull/49043) ([Rory Crispin](https://github.com/RoryCrispin)).
|
||||
* Allow parameters in queries with partitions like `ALTER TABLE t DROP PARTITION`. Closes [#49449](https://github.com/ClickHouse/ClickHouse/issues/49449). [#49516](https://github.com/ClickHouse/ClickHouse/pull/49516) ([Nikolay Degterinsky](https://github.com/evillique)).
|
||||
* 1.Refactor the code about zookeeper_connection 2.Add a new column xid for zookeeper_connection. [#50702](https://github.com/ClickHouse/ClickHouse/pull/50702) ([helifu](https://github.com/helifu)).
|
||||
|
@ -73,7 +73,7 @@ sidebar_label: 2023
|
||||
* ``` sumIf(123, cond) -> 123 * countIf(1, cond) sum(if(cond, 123, 0)) -> 123 * countIf(cond) sum(if(cond, 0, 123)) -> 123 * countIf(not(cond)) ```. [#44728](https://github.com/ClickHouse/ClickHouse/pull/44728) ([李扬](https://github.com/taiyang-li)).
|
||||
* Optimize behavior for a replica delay api logic in case the replica is read-only. [#45148](https://github.com/ClickHouse/ClickHouse/pull/45148) ([mateng915](https://github.com/mateng0915)).
|
||||
* Introduce gwp-asan implemented by llvm runtime. This closes [#27039](https://github.com/ClickHouse/ClickHouse/issues/27039). [#45226](https://github.com/ClickHouse/ClickHouse/pull/45226) ([Han Fei](https://github.com/hanfei1991)).
|
||||
* ... in the case key casted from uint64 to uint32, small impact for little endian platform but key value becomes zero in big endian case. ### Documentation entry for user-facing changes. [#45375](https://github.com/ClickHouse/ClickHouse/pull/45375) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
|
||||
* ... in the case key cast from uint64 to uint32, small impact for little endian platform but key value becomes zero in big endian case. ### Documentation entry for user-facing changes. [#45375](https://github.com/ClickHouse/ClickHouse/pull/45375) ([Suzy Wang](https://github.com/SuzyWangIBMer)).
|
||||
* Mark Gorilla compression on columns of non-Float* type as suspicious. [#45376](https://github.com/ClickHouse/ClickHouse/pull/45376) ([Robert Schulze](https://github.com/rschu1ze)).
|
||||
* Allow removing redundant aggregation keys with constants (e.g., simplify `GROUP BY a, a + 1` to `GROUP BY a`). [#45415](https://github.com/ClickHouse/ClickHouse/pull/45415) ([Dmitry Novik](https://github.com/novikd)).
|
||||
* Show replica name that is executing a merge in the postpone_reason. [#45458](https://github.com/ClickHouse/ClickHouse/pull/45458) ([Frank Chen](https://github.com/FrankChen021)).
|
||||
|
@ -15,7 +15,7 @@ sidebar_label: 2024
|
||||
* The system table `text_log` is enabled by default. This is fully compatible with previous versions, but you may notice subtly increased disk usage on the local disk (this system table takes a tiny amount of disk space). [#67428](https://github.com/ClickHouse/ClickHouse/pull/67428) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* In previous versions, `arrayWithConstant` can be slow if asked to generate very large arrays. In the new version, it is limited to 1 GB per array. This closes [#32754](https://github.com/ClickHouse/ClickHouse/issues/32754). [#67741](https://github.com/ClickHouse/ClickHouse/pull/67741) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Fix REPLACE modifier formatting (forbid omitting brackets). [#67774](https://github.com/ClickHouse/ClickHouse/pull/67774) ([Azat Khuzhin](https://github.com/azat)).
|
||||
* Backported in [#68349](https://github.com/ClickHouse/ClickHouse/issues/68349): Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
* Backported in [#68349](https://github.com/ClickHouse/ClickHouse/issues/68349): Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not cast to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Kruglov Pavel](https://github.com/Avogar)).
|
||||
|
||||
#### New Feature
|
||||
* Add new experimental Kafka storage engine to store offsets in Keeper instead of relying on committing them to Kafka. [#57625](https://github.com/ClickHouse/ClickHouse/pull/57625) ([János Benjamin Antal](https://github.com/antaljanosbenjamin)).
|
||||
|
@ -10,7 +10,7 @@ sidebar_label: 2024
|
||||
#### Backward Incompatible Change
|
||||
* Allow to write `SETTINGS` before `FORMAT` in a chain of queries with `UNION` when subqueries are inside parentheses. This closes [#39712](https://github.com/ClickHouse/ClickHouse/issues/39712). Change the behavior when a query has the SETTINGS clause specified twice in a sequence. The closest SETTINGS clause will have a preference for the corresponding subquery. In the previous versions, the outermost SETTINGS clause could take a preference over the inner one. [#60197](https://github.com/ClickHouse/ClickHouse/pull/60197) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* Do not allow explicitly specifying UUID when creating a table in Replicated database. Also, do not allow explicitly specifying ZooKeeper path and replica name for *MergeTree tables in Replicated databases. [#66104](https://github.com/ClickHouse/ClickHouse/pull/66104) ([Alexander Tokmakov](https://github.com/tavplubix)).
|
||||
* Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not casted to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Pavel Kruglov](https://github.com/Avogar)).
|
||||
* Reimplement Dynamic type. Now when the limit of dynamic data types is reached new types are not cast to String but stored in a special data structure in binary format with binary encoded data type. Now any type ever inserted into Dynamic column can be read from it as subcolumn. [#68132](https://github.com/ClickHouse/ClickHouse/pull/68132) ([Pavel Kruglov](https://github.com/Avogar)).
|
||||
* Expressions like `a[b].c` are supported for named tuples, as well as named subscripts from arbitrary expressions, e.g., `expr().name`. This is useful for processing JSON. This closes [#54965](https://github.com/ClickHouse/ClickHouse/issues/54965). In previous versions, an expression of form `expr().name` was parsed as `tupleElement(expr(), name)`, and the query analyzer was searching for a column `name` rather than for the corresponding tuple element; while in the new version, it is changed to `tupleElement(expr(), 'name')`. In most cases, the previous version was not working, but it is possible to imagine a very unusual scenario when this change could lead to incompatibility: if you stored names of tuple elements in a column or an alias, that was named differently than the tuple element's name: `SELECT 'b' AS a, CAST([tuple(123)] AS 'Array(Tuple(b UInt8))') AS t, t[1].a`. It is very unlikely that you used such queries, but we still have to mark this change as potentially backward incompatible. [#68435](https://github.com/ClickHouse/ClickHouse/pull/68435) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
* When the setting `print_pretty_type_names` is enabled, it will print `Tuple` data type in a pretty form in `SHOW CREATE TABLE` statements, `formatQuery` function, and in the interactive mode in `clickhouse-client` and `clickhouse-local`. In previous versions, this setting was only applied to `DESCRIBE` queries and `toTypeName`. This closes [#65753](https://github.com/ClickHouse/ClickHouse/issues/65753). [#68492](https://github.com/ClickHouse/ClickHouse/pull/68492) ([Alexey Milovidov](https://github.com/alexey-milovidov)).
|
||||
|
||||
|
@ -64,7 +64,7 @@ Result:
|
||||
|
||||
## Converting Tuple to Map
|
||||
|
||||
Values of type `Tuple()` can be casted to values of type `Map()` using function [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast):
|
||||
Values of type `Tuple()` can be cast to values of type `Map()` using function [CAST](../../sql-reference/functions/type-conversion-functions.md#type_conversion_function-cast):
|
||||
|
||||
**Example**
|
||||
|
||||
|
@ -187,7 +187,7 @@ select json.a.g.:Float64, dynamicType(json.a.g), json.d.:Date, dynamicType(json.
|
||||
└─────────────────────┴───────────────────────┴────────────────┴─────────────────────┘
|
||||
```
|
||||
|
||||
`Dynamic` subcolumns can be casted to any data type. In this case the exception will be thrown if internal type inside `Dynamic` cannot be casted to the requested type:
|
||||
`Dynamic` subcolumns can be cast to any data type. In this case the exception will be thrown if internal type inside `Dynamic` cannot be cast to the requested type:
|
||||
|
||||
```sql
|
||||
select json.a.g::UInt64 as uint FROM test;
|
||||
|
@ -8,7 +8,7 @@ sidebar_label: Arithmetic
|
||||
|
||||
Arithmetic functions work for any two operands of type `UInt8`, `UInt16`, `UInt32`, `UInt64`, `Int8`, `Int16`, `Int32`, `Int64`, `Float32`, or `Float64`.
|
||||
|
||||
Before performing the operation, both operands are casted to the result type. The result type is determined as follows (unless specified
|
||||
Before performing the operation, both operands are cast to the result type. The result type is determined as follows (unless specified
|
||||
differently in the function documentation below):
|
||||
- If both operands are up to 32 bits wide, the size of the result type will be the size of the next bigger type following the bigger of the
|
||||
two operands (integer size promotion). For example, `UInt8 + UInt16 = UInt32` or `Float32 * Float32 = Float64`.
|
||||
|
@ -85,7 +85,7 @@ Result:
|
||||
└───────────────────────────────────────────┘
|
||||
```
|
||||
|
||||
`mapFromArrays` also accepts arguments of type [Map](../data-types/map.md). These are casted to array of tuples during execution.
|
||||
`mapFromArrays` also accepts arguments of type [Map](../data-types/map.md). These are cast to array of tuples during execution.
|
||||
|
||||
```sql
|
||||
SELECT mapFromArrays([1, 2, 3], map('a', 1, 'b', 2, 'c', 3))
|
||||
|
@ -6257,7 +6257,7 @@ Code: 70. DB::Exception: Received from localhost:9000. DB::Exception: Value in c
|
||||
|
||||
## accurateCastOrNull(x, T)
|
||||
|
||||
Converts input value `x` to the specified data type `T`. Always returns [Nullable](../data-types/nullable.md) type and returns [NULL](../syntax.md/#null-literal) if the casted value is not representable in the target type.
|
||||
Converts input value `x` to the specified data type `T`. Always returns [Nullable](../data-types/nullable.md) type and returns [NULL](../syntax.md/#null-literal) if the cast value is not representable in the target type.
|
||||
|
||||
**Syntax**
|
||||
|
||||
@ -6310,7 +6310,7 @@ Result:
|
||||
|
||||
## accurateCastOrDefault(x, T[, default_value])
|
||||
|
||||
Converts input value `x` to the specified data type `T`. Returns default type value or `default_value` if specified if the casted value is not representable in the target type.
|
||||
Converts input value `x` to the specified data type `T`. Returns default type value or `default_value` if specified if the cast value is not representable in the target type.
|
||||
|
||||
**Syntax**
|
||||
|
||||
|
@ -21,4 +21,4 @@ Queries will add or remove metadata about constraints from table, so they are pr
|
||||
Constraint check **will not be executed** on existing data if it was added.
|
||||
:::
|
||||
|
||||
All changes on replicated tables are broadcasted to ZooKeeper and will be applied on other replicas as well.
|
||||
All changes on replicated tables are broadcast to ZooKeeper and will be applied on other replicas as well.
|
||||
|
@ -16,7 +16,7 @@ Manipulates data matching the specified filtering expression. Implemented as a [
|
||||
The `ALTER TABLE` prefix makes this syntax different from most other systems supporting SQL. It is intended to signify that unlike similar queries in OLTP databases this is a heavy operation not designed for frequent use.
|
||||
:::
|
||||
|
||||
The `filter_expr` must be of type `UInt8`. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are casted to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported.
|
||||
The `filter_expr` must be of type `UInt8`. This query updates values of specified columns to the values of corresponding expressions in rows for which the `filter_expr` takes a non-zero value. Values are cast to the column type using the `CAST` operator. Updating columns that are used in the calculation of the primary or the partition key is not supported.
|
||||
|
||||
One query can contain several commands separated by commas.
|
||||
|
||||
|
@ -68,7 +68,7 @@ struct ExternalDictionaryLibraryAPI
|
||||
using LibrarySettings = CStrings *;
|
||||
using LibraryData = void *;
|
||||
using RawClickHouseLibraryTable = void *;
|
||||
/// Can be safely casted into const Table * with static_cast<const ClickHouseLibrary::Table *>
|
||||
/// Can be safely cast into const Table * with static_cast<const ClickHouseLibrary::Table *>
|
||||
using RequestedColumnsNames = CStrings *;
|
||||
using RequestedIds = const VectorUInt64 *;
|
||||
using RequestedKeys = Table *;
|
||||
|
@ -384,7 +384,7 @@ public:
|
||||
auto * column = typeid_cast<ColumnFloat64 *>(&to);
|
||||
if (!column)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cast of column of predictions is incorrect. "
|
||||
"getReturnTypeToPredict must return same value as it is casted to");
|
||||
"getReturnTypeToPredict must return same value as it is cast to");
|
||||
|
||||
this->data(place).predict(column->getData(), arguments, offset, limit, context);
|
||||
}
|
||||
|
@ -174,7 +174,7 @@ struct IdentifierResolveScope
|
||||
size_t found_nullable_group_by_key_in_scope = 0;
|
||||
|
||||
/** It's possible that after a JOIN, a column in the projection has a type different from the column in the source table.
|
||||
* (For example, after join_use_nulls or USING column casted to supertype)
|
||||
* (For example, after join_use_nulls or USING column cast to supertype)
|
||||
* However, the column in the projection still refers to the table as its source.
|
||||
* This map is used to revert these columns back to their original columns in the source table.
|
||||
*/
|
||||
|
@ -392,14 +392,14 @@ void ColumnObjectDeprecated::Subcolumn::insertRangeFrom(const Subcolumn & src, s
|
||||
|
||||
if (n * 3 >= column->size())
|
||||
{
|
||||
auto casted_column = castColumn({column, column_type, ""}, least_common_type.get());
|
||||
data.back()->insertRangeFrom(*casted_column, from, n);
|
||||
auto cast_column = castColumn({column, column_type, ""}, least_common_type.get());
|
||||
data.back()->insertRangeFrom(*cast_column, from, n);
|
||||
return;
|
||||
}
|
||||
|
||||
auto casted_column = column->cut(from, n);
|
||||
casted_column = castColumn({casted_column, column_type, ""}, least_common_type.get());
|
||||
data.back()->insertRangeFrom(*casted_column, 0, n);
|
||||
auto cast_column = column->cut(from, n);
|
||||
cast_column = castColumn({cast_column, column_type, ""}, least_common_type.get());
|
||||
data.back()->insertRangeFrom(*cast_column, 0, n);
|
||||
};
|
||||
|
||||
size_t pos = 0;
|
||||
|
@ -156,7 +156,7 @@ public:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method get64 is not supported for {}", getName());
|
||||
}
|
||||
|
||||
/// If column stores native numeric type, it returns n-th element casted to Float64
|
||||
/// If column stores native numeric type, it returns n-th element cast to Float64
|
||||
/// Is used in regression methods to cast each features into uniform type
|
||||
[[nodiscard]] virtual Float64 getFloat64(size_t /*n*/) const
|
||||
{
|
||||
@ -168,7 +168,7 @@ public:
|
||||
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method getFloat32 is not supported for {}", getName());
|
||||
}
|
||||
|
||||
/** If column is numeric, return value of n-th element, casted to UInt64.
|
||||
/** If column is numeric, return value of n-th element, cast to UInt64.
|
||||
* For NULL values of Nullable column it is allowed to return arbitrary value.
|
||||
* Otherwise throw an exception.
|
||||
*/
|
||||
@ -185,7 +185,7 @@ public:
|
||||
[[nodiscard]] virtual bool isDefaultAt(size_t n) const = 0;
|
||||
[[nodiscard]] virtual bool isNullAt(size_t /*n*/) const { return false; }
|
||||
|
||||
/** If column is numeric, return value of n-th element, casted to bool.
|
||||
/** If column is numeric, return value of n-th element, cast to bool.
|
||||
* For NULL values of Nullable column returns false.
|
||||
* Otherwise throw an exception.
|
||||
*/
|
||||
|
@ -10,7 +10,7 @@
|
||||
/** Stores a calendar date in broken-down form (year, month, day-in-month).
|
||||
* Could be initialized from date in text form, like '2011-01-01' or from time_t with rounding to date.
|
||||
* Also could be initialized from date in text form like '20110101... (only first 8 symbols are used).
|
||||
* Could be implicitly casted to time_t.
|
||||
* Could be implicitly cast to time_t.
|
||||
* NOTE: Transforming between time_t and LocalDate is done in local time zone!
|
||||
*
|
||||
* When local time was shifted backwards (due to daylight saving time or whatever reason)
|
||||
|
@ -9,7 +9,7 @@
|
||||
|
||||
/** Stores calendar date and time in broken-down form.
|
||||
* Could be initialized from date and time in text form like '2011-01-01 00:00:00' or from time_t.
|
||||
* Could be implicitly casted to time_t.
|
||||
* Could be implicitly cast to time_t.
|
||||
* NOTE: Transforming between time_t and LocalDate is done in local time zone!
|
||||
*
|
||||
* When local time was shifted backwards (due to daylight saving time or whatever reason)
|
||||
|
@ -66,7 +66,7 @@ private:
|
||||
#ifdef __SSE4_1__
|
||||
/// second character of "needle" (if its length is > 1)
|
||||
uint8_t second_needle_character = 0;
|
||||
/// first/second needle character broadcasted into a 16 bytes vector
|
||||
/// first/second needle character broadcast into a 16 bytes vector
|
||||
__m128i first_needle_character_vec;
|
||||
__m128i second_needle_character_vec;
|
||||
/// vector of first 16 characters of `needle`
|
||||
@ -204,8 +204,8 @@ public:
|
||||
while (haystack < haystack_end && haystack_end - haystack >= needle_size)
|
||||
{
|
||||
#ifdef __SSE4_1__
|
||||
/// Compare the [0:15] bytes from haystack and broadcasted 16 bytes vector from first character of needle.
|
||||
/// Compare the [1:16] bytes from haystack and broadcasted 16 bytes vector from second character of needle.
|
||||
/// Compare the [0:15] bytes from haystack and broadcast 16 bytes vector from first character of needle.
|
||||
/// Compare the [1:16] bytes from haystack and broadcast 16 bytes vector from second character of needle.
|
||||
/// Bit AND the results of above two comparisons and get the mask.
|
||||
if ((haystack + 1 + N) <= haystack_end && isPageSafe(haystack + 1))
|
||||
{
|
||||
|
@ -182,7 +182,7 @@ bool waitForPid(pid_t pid, size_t timeout_in_seconds)
|
||||
/// If timeout is positive try waitpid without block in loop until
|
||||
/// process is normally terminated or waitpid return error
|
||||
|
||||
/// NOTE: timeout casted to int, since poll() accept int for timeout
|
||||
/// NOTE: timeout cast to int, since poll() accept int for timeout
|
||||
int timeout_in_ms = static_cast<int>(timeout_in_seconds * 1000);
|
||||
while (timeout_in_ms > 0)
|
||||
{
|
||||
|
@ -514,12 +514,12 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
|
||||
nuraft::ptr<nuraft::delayed_task_scheduler> scheduler = asio_service;
|
||||
nuraft::ptr<nuraft::rpc_client_factory> rpc_cli_factory = asio_service;
|
||||
|
||||
nuraft::ptr<nuraft::state_mgr> casted_state_manager = state_manager;
|
||||
nuraft::ptr<nuraft::state_machine> casted_state_machine = state_machine;
|
||||
nuraft::ptr<nuraft::state_mgr> cast_state_manager = state_manager;
|
||||
nuraft::ptr<nuraft::state_machine> cast_state_machine = state_machine;
|
||||
|
||||
/// raft_server creates unique_ptr from it
|
||||
nuraft::context * ctx
|
||||
= new nuraft::context(casted_state_manager, casted_state_machine, asio_listeners, logger, rpc_cli_factory, scheduler, params);
|
||||
= new nuraft::context(cast_state_manager, cast_state_machine, asio_listeners, logger, rpc_cli_factory, scheduler, params);
|
||||
|
||||
raft_instance = nuraft::cs_new<KeeperRaftServer>(ctx, init_options);
|
||||
|
||||
@ -535,11 +535,11 @@ void KeeperServer::launchRaftServer(const Poco::Util::AbstractConfiguration & co
|
||||
|
||||
raft_instance->start_server(init_options.skip_initial_election_timeout_);
|
||||
|
||||
nuraft::ptr<nuraft::raft_server> casted_raft_server = raft_instance;
|
||||
nuraft::ptr<nuraft::raft_server> cast_raft_server = raft_instance;
|
||||
|
||||
for (const auto & asio_listener : asio_listeners)
|
||||
{
|
||||
asio_listener->listen(casted_raft_server);
|
||||
asio_listener->listen(cast_raft_server);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -120,7 +120,7 @@ using SortDescriptionWithPositions = std::vector<SortColumnDescriptionWithColumn
|
||||
class SortDescription : public std::vector<SortColumnDescription>
|
||||
{
|
||||
public:
|
||||
/// Can be safely casted into JITSortDescriptionFunc
|
||||
/// Can be safely cast into JITSortDescriptionFunc
|
||||
void * compiled_sort_description = nullptr;
|
||||
std::shared_ptr<CompiledSortDescriptionFunctionHolder> compiled_sort_description_holder;
|
||||
size_t min_count_to_compile_sort_description = 3;
|
||||
|
@ -205,7 +205,7 @@ struct ResultOfIf
|
||||
ConstructedType, Error>>>;
|
||||
};
|
||||
|
||||
/** Before applying operator `%` and bitwise operations, operands are casted to whole numbers. */
|
||||
/** Before applying operator `%` and bitwise operations, operands are cast to whole numbers. */
|
||||
template <typename A> struct ToInteger
|
||||
{
|
||||
using Type = typename Construct<
|
||||
|
@ -9,7 +9,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type)
|
||||
bool canBeSafelyCast(const DataTypePtr & from_type, const DataTypePtr & to_type)
|
||||
{
|
||||
auto from_which_type = WhichDataType(from_type->getTypeId());
|
||||
bool to_type_was_nullable = isNullableOrLowCardinalityNullable(to_type);
|
||||
@ -127,7 +127,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
if (to_type_was_nullable)
|
||||
{
|
||||
const auto & from_type_nullable = assert_cast<const DataTypeNullable &>(*from_type);
|
||||
return canBeSafelyCasted(from_type_nullable.getNestedType(), to_type_unwrapped);
|
||||
return canBeSafelyCast(from_type_nullable.getNestedType(), to_type_unwrapped);
|
||||
}
|
||||
|
||||
if (to_which_type.isString())
|
||||
@ -138,7 +138,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
case TypeIndex::LowCardinality:
|
||||
{
|
||||
const auto & from_type_low_cardinality = assert_cast<const DataTypeLowCardinality &>(*from_type);
|
||||
return canBeSafelyCasted(from_type_low_cardinality.getDictionaryType(), to_type_unwrapped);
|
||||
return canBeSafelyCast(from_type_low_cardinality.getDictionaryType(), to_type_unwrapped);
|
||||
}
|
||||
case TypeIndex::Array:
|
||||
{
|
||||
@ -146,7 +146,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
{
|
||||
const auto & from_type_array = assert_cast<const DataTypeArray &>(*from_type);
|
||||
const auto & to_type_array = assert_cast<const DataTypeArray &>(*to_type_unwrapped);
|
||||
return canBeSafelyCasted(from_type_array.getNestedType(), to_type_array.getNestedType());
|
||||
return canBeSafelyCast(from_type_array.getNestedType(), to_type_array.getNestedType());
|
||||
}
|
||||
|
||||
if (to_which_type.isString())
|
||||
@ -160,10 +160,10 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
{
|
||||
const auto & from_type_map = assert_cast<const DataTypeMap &>(*from_type);
|
||||
const auto & to_type_map = assert_cast<const DataTypeMap &>(*to_type_unwrapped);
|
||||
if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_map.getKeyType()))
|
||||
if (!canBeSafelyCast(from_type_map.getKeyType(), to_type_map.getKeyType()))
|
||||
return false;
|
||||
|
||||
if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_map.getValueType()))
|
||||
if (!canBeSafelyCast(from_type_map.getValueType(), to_type_map.getValueType()))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
@ -182,10 +182,10 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
if (to_type_tuple_elements.size() != 2)
|
||||
return false;
|
||||
|
||||
if (!canBeSafelyCasted(from_type_map.getKeyType(), to_type_tuple_elements[0]))
|
||||
if (!canBeSafelyCast(from_type_map.getKeyType(), to_type_tuple_elements[0]))
|
||||
return false;
|
||||
|
||||
if (!canBeSafelyCasted(from_type_map.getValueType(), to_type_tuple_elements[1]))
|
||||
if (!canBeSafelyCast(from_type_map.getValueType(), to_type_tuple_elements[1]))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
@ -211,7 +211,7 @@ bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_typ
|
||||
return false;
|
||||
|
||||
for (size_t i = 0; i < lhs_type_elements_size; ++i)
|
||||
if (!canBeSafelyCasted(from_tuple_type_elements[i], to_tuple_type_elements[i]))
|
||||
if (!canBeSafelyCast(from_tuple_type_elements[i], to_tuple_type_elements[i]))
|
||||
return false;
|
||||
|
||||
return true;
|
||||
|
@ -5,7 +5,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/** Returns true if from_type can be safely casted to to_type.
|
||||
/** Returns true if from_type can be safely cast to to_type.
|
||||
*
|
||||
* Examples:
|
||||
* From type UInt8 to type UInt16 returns true.
|
||||
@ -14,6 +14,6 @@ namespace DB
|
||||
* From type LowCardinality(String) to type String returns true.
|
||||
* From type String to type UInt8 returns false.
|
||||
*/
|
||||
bool canBeSafelyCasted(const DataTypePtr & from_type, const DataTypePtr & to_type);
|
||||
bool canBeSafelyCast(const DataTypePtr & from_type, const DataTypePtr & to_type);
|
||||
|
||||
}
|
||||
|
@ -24,11 +24,11 @@ template <LeastSupertypeOnError on_error = LeastSupertypeOnError::Throw>
|
||||
DataTypePtr getLeastSupertype(const DataTypes & types);
|
||||
|
||||
/// Same as above but return String type instead of throwing exception.
|
||||
/// All types can be casted to String, because they can be serialized to String.
|
||||
/// All types can be cast to String, because they can be serialized to String.
|
||||
DataTypePtr getLeastSupertypeOrString(const DataTypes & types);
|
||||
|
||||
/// Same as getLeastSupertype but in case when there is no supertype for provided types
|
||||
/// it uses Variant of these types as a supertype. Any type can be casted to a Variant
|
||||
/// it uses Variant of these types as a supertype. Any type can be cast to a Variant
|
||||
/// that contains this type.
|
||||
/// As nested Variants are not allowed, if one of the types is Variant, it's variants
|
||||
/// are used in the resulting Variant.
|
||||
|
@ -751,7 +751,7 @@ static void writeFieldsToColumn(
|
||||
return true;
|
||||
};
|
||||
|
||||
const auto & write_data_to_column = [&](auto * casted_column, auto from_type, auto to_type)
|
||||
const auto & write_data_to_column = [&](auto * cast_column, auto from_type, auto to_type)
|
||||
{
|
||||
for (size_t index = 0; index < rows_data.size(); ++index)
|
||||
{
|
||||
@ -759,39 +759,39 @@ static void writeFieldsToColumn(
|
||||
const Field & value = row_data[column_index];
|
||||
|
||||
if (write_data_to_null_map(value, index))
|
||||
casted_column->insertValue(static_cast<decltype(to_type)>(value.template safeGet<decltype(from_type)>()));
|
||||
cast_column->insertValue(static_cast<decltype(to_type)>(value.template safeGet<decltype(from_type)>()));
|
||||
}
|
||||
};
|
||||
|
||||
if (ColumnInt8 * casted_int8_column = typeid_cast<ColumnInt8 *>(&column_to))
|
||||
write_data_to_column(casted_int8_column, UInt64(), Int8());
|
||||
else if (ColumnInt16 * casted_int16_column = typeid_cast<ColumnInt16 *>(&column_to))
|
||||
write_data_to_column(casted_int16_column, UInt64(), Int16());
|
||||
else if (ColumnInt64 * casted_int64_column = typeid_cast<ColumnInt64 *>(&column_to))
|
||||
write_data_to_column(casted_int64_column, UInt64(), Int64());
|
||||
else if (ColumnUInt8 * casted_uint8_column = typeid_cast<ColumnUInt8 *>(&column_to))
|
||||
write_data_to_column(casted_uint8_column, UInt64(), UInt8());
|
||||
else if (ColumnUInt16 * casted_uint16_column = typeid_cast<ColumnUInt16 *>(&column_to))
|
||||
write_data_to_column(casted_uint16_column, UInt64(), UInt16());
|
||||
else if (ColumnUInt32 * casted_uint32_column = typeid_cast<ColumnUInt32 *>(&column_to))
|
||||
write_data_to_column(casted_uint32_column, UInt64(), UInt32());
|
||||
else if (ColumnUInt64 * casted_uint64_column = typeid_cast<ColumnUInt64 *>(&column_to))
|
||||
write_data_to_column(casted_uint64_column, UInt64(), UInt64());
|
||||
else if (ColumnFloat32 * casted_float32_column = typeid_cast<ColumnFloat32 *>(&column_to))
|
||||
write_data_to_column(casted_float32_column, Float64(), Float32());
|
||||
else if (ColumnFloat64 * casted_float64_column = typeid_cast<ColumnFloat64 *>(&column_to))
|
||||
write_data_to_column(casted_float64_column, Float64(), Float64());
|
||||
else if (ColumnDecimal<Decimal32> * casted_decimal_32_column = typeid_cast<ColumnDecimal<Decimal32> *>(&column_to))
|
||||
write_data_to_column(casted_decimal_32_column, Decimal32(), Decimal32());
|
||||
else if (ColumnDecimal<Decimal64> * casted_decimal_64_column = typeid_cast<ColumnDecimal<Decimal64> *>(&column_to))
|
||||
write_data_to_column(casted_decimal_64_column, Decimal64(), Decimal64());
|
||||
else if (ColumnDecimal<Decimal128> * casted_decimal_128_column = typeid_cast<ColumnDecimal<Decimal128> *>(&column_to))
|
||||
write_data_to_column(casted_decimal_128_column, Decimal128(), Decimal128());
|
||||
else if (ColumnDecimal<Decimal256> * casted_decimal_256_column = typeid_cast<ColumnDecimal<Decimal256> *>(&column_to))
|
||||
write_data_to_column(casted_decimal_256_column, Decimal256(), Decimal256());
|
||||
else if (ColumnDecimal<DateTime64> * casted_datetime_64_column = typeid_cast<ColumnDecimal<DateTime64> *>(&column_to))
|
||||
write_data_to_column(casted_datetime_64_column, DateTime64(), DateTime64());
|
||||
else if (ColumnInt32 * casted_int32_column = typeid_cast<ColumnInt32 *>(&column_to))
|
||||
if (ColumnInt8 * cast_int8_column = typeid_cast<ColumnInt8 *>(&column_to))
|
||||
write_data_to_column(cast_int8_column, UInt64(), Int8());
|
||||
else if (ColumnInt16 * cast_int16_column = typeid_cast<ColumnInt16 *>(&column_to))
|
||||
write_data_to_column(cast_int16_column, UInt64(), Int16());
|
||||
else if (ColumnInt64 * cast_int64_column = typeid_cast<ColumnInt64 *>(&column_to))
|
||||
write_data_to_column(cast_int64_column, UInt64(), Int64());
|
||||
else if (ColumnUInt8 * cast_uint8_column = typeid_cast<ColumnUInt8 *>(&column_to))
|
||||
write_data_to_column(cast_uint8_column, UInt64(), UInt8());
|
||||
else if (ColumnUInt16 * cast_uint16_column = typeid_cast<ColumnUInt16 *>(&column_to))
|
||||
write_data_to_column(cast_uint16_column, UInt64(), UInt16());
|
||||
else if (ColumnUInt32 * cast_uint32_column = typeid_cast<ColumnUInt32 *>(&column_to))
|
||||
write_data_to_column(cast_uint32_column, UInt64(), UInt32());
|
||||
else if (ColumnUInt64 * cast_uint64_column = typeid_cast<ColumnUInt64 *>(&column_to))
|
||||
write_data_to_column(cast_uint64_column, UInt64(), UInt64());
|
||||
else if (ColumnFloat32 * cast_float32_column = typeid_cast<ColumnFloat32 *>(&column_to))
|
||||
write_data_to_column(cast_float32_column, Float64(), Float32());
|
||||
else if (ColumnFloat64 * cast_float64_column = typeid_cast<ColumnFloat64 *>(&column_to))
|
||||
write_data_to_column(cast_float64_column, Float64(), Float64());
|
||||
else if (ColumnDecimal<Decimal32> * cast_decimal_32_column = typeid_cast<ColumnDecimal<Decimal32> *>(&column_to))
|
||||
write_data_to_column(cast_decimal_32_column, Decimal32(), Decimal32());
|
||||
else if (ColumnDecimal<Decimal64> * cast_decimal_64_column = typeid_cast<ColumnDecimal<Decimal64> *>(&column_to))
|
||||
write_data_to_column(cast_decimal_64_column, Decimal64(), Decimal64());
|
||||
else if (ColumnDecimal<Decimal128> * cast_decimal_128_column = typeid_cast<ColumnDecimal<Decimal128> *>(&column_to))
|
||||
write_data_to_column(cast_decimal_128_column, Decimal128(), Decimal128());
|
||||
else if (ColumnDecimal<Decimal256> * cast_decimal_256_column = typeid_cast<ColumnDecimal<Decimal256> *>(&column_to))
|
||||
write_data_to_column(cast_decimal_256_column, Decimal256(), Decimal256());
|
||||
else if (ColumnDecimal<DateTime64> * cast_datetime_64_column = typeid_cast<ColumnDecimal<DateTime64> *>(&column_to))
|
||||
write_data_to_column(cast_datetime_64_column, DateTime64(), DateTime64());
|
||||
else if (ColumnInt32 * cast_int32_column = typeid_cast<ColumnInt32 *>(&column_to))
|
||||
{
|
||||
for (size_t index = 0; index < rows_data.size(); ++index)
|
||||
{
|
||||
@ -801,19 +801,19 @@ static void writeFieldsToColumn(
|
||||
if (write_data_to_null_map(value, index))
|
||||
{
|
||||
if (value.getType() == Field::Types::UInt64)
|
||||
casted_int32_column->insertValue(static_cast<Int32>(value.safeGet<Int32>()));
|
||||
cast_int32_column->insertValue(static_cast<Int32>(value.safeGet<Int32>()));
|
||||
else if (value.getType() == Field::Types::Int64)
|
||||
{
|
||||
/// For MYSQL_TYPE_INT24
|
||||
const Int32 & num = static_cast<Int32>(value.safeGet<Int32>());
|
||||
casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num);
|
||||
cast_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug.");
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (ColumnString * casted_string_column = typeid_cast<ColumnString *>(&column_to))
|
||||
else if (ColumnString * cast_string_column = typeid_cast<ColumnString *>(&column_to))
|
||||
{
|
||||
for (size_t index = 0; index < rows_data.size(); ++index)
|
||||
{
|
||||
@ -823,11 +823,11 @@ static void writeFieldsToColumn(
|
||||
if (write_data_to_null_map(value, index))
|
||||
{
|
||||
const String & data = value.safeGet<const String &>();
|
||||
casted_string_column->insertData(data.data(), data.size());
|
||||
cast_string_column->insertData(data.data(), data.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (ColumnFixedString * casted_fixed_string_column = typeid_cast<ColumnFixedString *>(&column_to))
|
||||
else if (ColumnFixedString * cast_fixed_string_column = typeid_cast<ColumnFixedString *>(&column_to))
|
||||
{
|
||||
for (size_t index = 0; index < rows_data.size(); ++index)
|
||||
{
|
||||
@ -837,7 +837,7 @@ static void writeFieldsToColumn(
|
||||
if (write_data_to_null_map(value, index))
|
||||
{
|
||||
const String & data = value.safeGet<const String &>();
|
||||
casted_fixed_string_column->insertData(data.data(), data.size());
|
||||
cast_fixed_string_column->insertData(data.data(), data.size());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -163,8 +163,8 @@ public:
|
||||
|
||||
auto & key_column_to_cast = key_columns[key_attribute_type_index];
|
||||
ColumnWithTypeAndName column_to_cast = {key_column_to_cast, key_type, ""};
|
||||
auto casted_column = castColumnAccurate(column_to_cast, key_attribute_type);
|
||||
key_column_to_cast = std::move(casted_column);
|
||||
auto cast_column = castColumnAccurate(column_to_cast, key_attribute_type);
|
||||
key_column_to_cast = std::move(cast_column);
|
||||
key_type = key_attribute_type;
|
||||
}
|
||||
}
|
||||
|
@ -62,8 +62,8 @@ void IPolygonDictionary::convertKeyColumns(Columns & key_columns, DataTypes & ke
|
||||
|
||||
auto & key_column_to_cast = key_columns[key_type_index];
|
||||
ColumnWithTypeAndName column_to_cast = {key_column_to_cast, key_type, ""};
|
||||
auto casted_column = castColumnAccurate(column_to_cast, float_64_type);
|
||||
key_column_to_cast = std::move(casted_column);
|
||||
auto cast_column = castColumnAccurate(column_to_cast, float_64_type);
|
||||
key_column_to_cast = std::move(cast_column);
|
||||
key_type = float_64_type;
|
||||
}
|
||||
}
|
||||
|
@ -227,33 +227,33 @@ namespace
|
||||
template <typename NumberType>
|
||||
void writeInt(NumberType value)
|
||||
{
|
||||
auto casted = castNumber<Int64>(value);
|
||||
if (casted != 0 || !skip_zero_or_empty)
|
||||
writer->writeInt(field_tag, casted);
|
||||
auto cast = castNumber<Int64>(value);
|
||||
if (cast != 0 || !skip_zero_or_empty)
|
||||
writer->writeInt(field_tag, cast);
|
||||
}
|
||||
|
||||
template <typename NumberType>
|
||||
void writeSInt(NumberType value)
|
||||
{
|
||||
auto casted = castNumber<Int64>(value);
|
||||
if (casted != 0 || !skip_zero_or_empty)
|
||||
writer->writeSInt(field_tag, casted);
|
||||
auto cast = castNumber<Int64>(value);
|
||||
if (cast != 0 || !skip_zero_or_empty)
|
||||
writer->writeSInt(field_tag, cast);
|
||||
}
|
||||
|
||||
template <typename NumberType>
|
||||
void writeUInt(NumberType value)
|
||||
{
|
||||
auto casted = castNumber<UInt64>(value);
|
||||
if (casted != 0 || !skip_zero_or_empty)
|
||||
writer->writeUInt(field_tag, casted);
|
||||
auto cast = castNumber<UInt64>(value);
|
||||
if (cast != 0 || !skip_zero_or_empty)
|
||||
writer->writeUInt(field_tag, cast);
|
||||
}
|
||||
|
||||
template <typename FieldType, typename NumberType>
|
||||
void writeFixed(NumberType value)
|
||||
{
|
||||
auto casted = castNumber<FieldType>(value);
|
||||
if (casted != 0 || !skip_zero_or_empty)
|
||||
writer->writeFixed(field_tag, casted);
|
||||
auto cast = castNumber<FieldType>(value);
|
||||
if (cast != 0 || !skip_zero_or_empty)
|
||||
writer->writeFixed(field_tag, cast);
|
||||
}
|
||||
|
||||
Int64 readInt() { return reader->readInt(); }
|
||||
|
@ -84,7 +84,7 @@ struct DivideIntegralImpl
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Comparisons are not strict to avoid rounding issues when operand is implicitly casted to float.
|
||||
/// Comparisons are not strict to avoid rounding issues when operand is implicitly cast to float.
|
||||
|
||||
if constexpr (is_floating_point<A>)
|
||||
if (isNaN(a) || a >= std::numeric_limits<CastA>::max() || a <= std::numeric_limits<CastA>::lowest())
|
||||
|
@ -206,7 +206,7 @@ ColumnPtr ExecutableFunctionDynamicAdaptor::executeImpl(const ColumnsWithTypeAnd
|
||||
nested_result_type = result_type;
|
||||
}
|
||||
|
||||
/// If the result of nested function is Dynamic (or we casted to it from Variant), we can just expand it
|
||||
/// If the result of nested function is Dynamic (or we cast to it from Variant), we can just expand it
|
||||
/// and cast to the result Dynamic type (it can have different max_types parameter).
|
||||
if (isDynamic(nested_result_type))
|
||||
{
|
||||
|
@ -4204,7 +4204,7 @@ private:
|
||||
{
|
||||
try
|
||||
{
|
||||
/// We can avoid try/catch here if we will implement check that 2 types can be casted, but it
|
||||
/// We can avoid try/catch here if we will implement check that 2 types can be cast, but it
|
||||
/// requires quite a lot of work. By now let's simply use try/catch.
|
||||
/// First, check that we can create a wrapper.
|
||||
WrapperType wrapper = prepareUnpackDictionaries(from, to);
|
||||
@ -4250,31 +4250,31 @@ private:
|
||||
const auto & column_variant = assert_cast<const ColumnVariant &>(*arguments.front().column.get());
|
||||
|
||||
/// First, cast each variant to the result type.
|
||||
std::vector<ColumnPtr> casted_variant_columns;
|
||||
casted_variant_columns.reserve(variant_types.size());
|
||||
std::vector<ColumnPtr> cast_variant_columns;
|
||||
cast_variant_columns.reserve(variant_types.size());
|
||||
for (size_t i = 0; i != variant_types.size(); ++i)
|
||||
{
|
||||
auto variant_col = column_variant.getVariantPtrByGlobalDiscriminator(i);
|
||||
ColumnsWithTypeAndName variant = {{variant_col, variant_types[i], "" }};
|
||||
const auto & variant_wrapper = variant_wrappers[i];
|
||||
ColumnPtr casted_variant;
|
||||
ColumnPtr cast_variant;
|
||||
/// Check if we have wrapper for this variant.
|
||||
if (variant_wrapper)
|
||||
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
|
||||
casted_variant_columns.push_back(std::move(casted_variant));
|
||||
cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
|
||||
cast_variant_columns.push_back(std::move(cast_variant));
|
||||
}
|
||||
|
||||
/// Second, construct resulting column from casted variant columns according to discriminators.
|
||||
/// Second, construct resulting column from cast variant columns according to discriminators.
|
||||
const auto & local_discriminators = column_variant.getLocalDiscriminators();
|
||||
auto res = result_type->createColumn();
|
||||
res->reserve(input_rows_count);
|
||||
for (size_t i = 0; i != input_rows_count; ++i)
|
||||
{
|
||||
auto global_discr = column_variant.globalDiscriminatorByLocal(local_discriminators[i]);
|
||||
if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !casted_variant_columns[global_discr])
|
||||
if (global_discr == ColumnVariant::NULL_DISCRIMINATOR || !cast_variant_columns[global_discr])
|
||||
res->insertDefault();
|
||||
else
|
||||
res->insertFrom(*casted_variant_columns[global_discr], column_variant.offsetAt(i));
|
||||
res->insertFrom(*cast_variant_columns[global_discr], column_variant.offsetAt(i));
|
||||
}
|
||||
|
||||
return res;
|
||||
@ -4457,14 +4457,14 @@ private:
|
||||
|
||||
/// First, cast usual variants to result type.
|
||||
const auto & variant_types = assert_cast<const DataTypeVariant &>(*variant_info.variant_type).getVariants();
|
||||
std::vector<ColumnPtr> casted_variant_columns;
|
||||
casted_variant_columns.reserve(variant_types.size());
|
||||
std::vector<ColumnPtr> cast_variant_columns;
|
||||
cast_variant_columns.reserve(variant_types.size());
|
||||
for (size_t i = 0; i != variant_types.size(); ++i)
|
||||
{
|
||||
/// Skip shared variant, it will be processed later.
|
||||
if (i == column_dynamic.getSharedVariantDiscriminator())
|
||||
{
|
||||
casted_variant_columns.push_back(nullptr);
|
||||
cast_variant_columns.push_back(nullptr);
|
||||
continue;
|
||||
}
|
||||
|
||||
@ -4477,11 +4477,11 @@ private:
|
||||
else
|
||||
variant_wrapper = prepareUnpackDictionaries(variant_types[i], result_type);
|
||||
|
||||
ColumnPtr casted_variant;
|
||||
ColumnPtr cast_variant;
|
||||
/// Check if we have wrapper for this variant.
|
||||
if (variant_wrapper)
|
||||
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
|
||||
casted_variant_columns.push_back(casted_variant);
|
||||
cast_variant = variant_wrapper(variant, result_type, nullptr, variant_col->size());
|
||||
cast_variant_columns.push_back(cast_variant);
|
||||
}
|
||||
|
||||
/// Second, collect all variants stored in shared variant and cast them to result type.
|
||||
@ -4532,8 +4532,8 @@ private:
|
||||
}
|
||||
|
||||
/// Cast all extracted variants into result type.
|
||||
std::vector<ColumnPtr> casted_shared_variant_columns;
|
||||
casted_shared_variant_columns.reserve(variant_types_from_shared_variant.size());
|
||||
std::vector<ColumnPtr> cast_shared_variant_columns;
|
||||
cast_shared_variant_columns.reserve(variant_types_from_shared_variant.size());
|
||||
for (size_t i = 0; i != variant_types_from_shared_variant.size(); ++i)
|
||||
{
|
||||
ColumnsWithTypeAndName variant = {{variant_columns_from_shared_variant[i]->getPtr(), variant_types_from_shared_variant[i], ""}};
|
||||
@ -4544,14 +4544,14 @@ private:
|
||||
else
|
||||
variant_wrapper = prepareUnpackDictionaries(variant_types_from_shared_variant[i], result_type);
|
||||
|
||||
ColumnPtr casted_variant;
|
||||
ColumnPtr cast_variant;
|
||||
/// Check if we have wrapper for this variant.
|
||||
if (variant_wrapper)
|
||||
casted_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size());
|
||||
casted_shared_variant_columns.push_back(casted_variant);
|
||||
cast_variant = variant_wrapper(variant, result_type, nullptr, variant_columns_from_shared_variant[i]->size());
|
||||
cast_shared_variant_columns.push_back(cast_variant);
|
||||
}
|
||||
|
||||
/// Construct result column from all casted variants.
|
||||
/// Construct result column from all cast variants.
|
||||
auto res = result_type->createColumn();
|
||||
res->reserve(input_rows_count);
|
||||
for (size_t i = 0; i != input_rows_count; ++i)
|
||||
@ -4563,15 +4563,15 @@ private:
|
||||
}
|
||||
else if (global_discr == shared_variant_discr)
|
||||
{
|
||||
if (casted_shared_variant_columns[shared_variant_indexes[i]])
|
||||
res->insertFrom(*casted_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]);
|
||||
if (cast_shared_variant_columns[shared_variant_indexes[i]])
|
||||
res->insertFrom(*cast_shared_variant_columns[shared_variant_indexes[i]], shared_variant_offsets[i]);
|
||||
else
|
||||
res->insertDefault();
|
||||
}
|
||||
else
|
||||
{
|
||||
if (casted_variant_columns[global_discr])
|
||||
res->insertFrom(*casted_variant_columns[global_discr], offsets[i]);
|
||||
if (cast_variant_columns[global_discr])
|
||||
res->insertFrom(*cast_variant_columns[global_discr], offsets[i]);
|
||||
else
|
||||
res->insertDefault();
|
||||
}
|
||||
@ -5032,7 +5032,7 @@ private:
|
||||
ColumnPtr converted_column;
|
||||
|
||||
ColumnPtr res_indexes;
|
||||
/// For some types default can't be casted (for example, String to Int). In that case convert column to full.
|
||||
/// For some types default can't be cast (for example, String to Int). In that case convert column to full.
|
||||
bool src_converted_to_full_column = false;
|
||||
|
||||
{
|
||||
|
@ -622,11 +622,11 @@ private:
|
||||
column_before_cast.type,
|
||||
column_before_cast.name};
|
||||
|
||||
auto casted = IColumn::mutate(castColumnAccurate(column_to_cast, result_type));
|
||||
auto cast = IColumn::mutate(castColumnAccurate(column_to_cast, result_type));
|
||||
|
||||
auto mask_col = ColumnUInt8::create();
|
||||
mask_col->getData() = std::move(default_mask);
|
||||
return {std::move(casted), std::move(mask_col)};
|
||||
return {std::move(cast), std::move(mask_col)};
|
||||
}
|
||||
|
||||
void restoreShortCircuitColumn(
|
||||
@ -1124,9 +1124,9 @@ private:
|
||||
const auto & hierarchical_attribute = FunctionDictHelper::getDictionaryHierarchicalAttribute(dictionary);
|
||||
|
||||
auto key_column = ColumnWithTypeAndName{arguments[1].column, arguments[1].type, arguments[1].name};
|
||||
auto key_column_casted = castColumnAccurate(key_column, removeNullable(hierarchical_attribute.type));
|
||||
auto key_column_cast = castColumnAccurate(key_column, removeNullable(hierarchical_attribute.type));
|
||||
|
||||
ColumnPtr result = dictionary->getHierarchy(key_column_casted, hierarchical_attribute.type);
|
||||
ColumnPtr result = dictionary->getHierarchy(key_column_cast, hierarchical_attribute.type);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -1182,10 +1182,10 @@ private:
|
||||
auto in_key_column = ColumnWithTypeAndName{arguments[2].column->convertToFullColumnIfConst(), arguments[2].type, arguments[2].name};
|
||||
|
||||
auto hierarchical_attribute_non_nullable = removeNullable(hierarchical_attribute.type);
|
||||
auto key_column_casted = castColumnAccurate(key_column, hierarchical_attribute_non_nullable);
|
||||
auto in_key_column_casted = castColumnAccurate(in_key_column, hierarchical_attribute_non_nullable);
|
||||
auto key_column_cast = castColumnAccurate(key_column, hierarchical_attribute_non_nullable);
|
||||
auto in_key_column_cast = castColumnAccurate(in_key_column, hierarchical_attribute_non_nullable);
|
||||
|
||||
ColumnPtr result = dictionary->isInHierarchy(key_column_casted, in_key_column_casted, hierarchical_attribute.type);
|
||||
ColumnPtr result = dictionary->isInHierarchy(key_column_cast, in_key_column_cast, hierarchical_attribute.type);
|
||||
|
||||
return result;
|
||||
}
|
||||
@ -1222,9 +1222,9 @@ public:
|
||||
const auto & hierarchical_attribute = dictionary_helper->getDictionaryHierarchicalAttribute(dictionary);
|
||||
|
||||
auto key_column = ColumnWithTypeAndName{arguments[1].column->convertToFullColumnIfConst(), arguments[1].type, arguments[1].name};
|
||||
auto key_column_casted = castColumnAccurate(key_column, removeNullable(hierarchical_attribute.type));
|
||||
auto key_column_cast = castColumnAccurate(key_column, removeNullable(hierarchical_attribute.type));
|
||||
|
||||
return dictionary->getDescendants(key_column_casted, removeNullable(hierarchical_attribute.type), level, hierarchical_parent_to_child_index);
|
||||
return dictionary->getDescendants(key_column_cast, removeNullable(hierarchical_attribute.type), level, hierarchical_parent_to_child_index);
|
||||
}
|
||||
|
||||
String name;
|
||||
|
@ -100,7 +100,7 @@ private:
|
||||
struct CastArgumentsResult
|
||||
{
|
||||
ColumnsWithTypeAndName initial;
|
||||
ColumnsWithTypeAndName casted;
|
||||
ColumnsWithTypeAndName cast;
|
||||
};
|
||||
|
||||
static CastArgumentsResult castColumns(const ColumnsWithTypeAndName & arguments,
|
||||
@ -214,8 +214,8 @@ ColumnPtr FunctionArrayIntersect<Mode>::castRemoveNullable(const ColumnPtr & col
|
||||
const auto & nested = column_nullable->getNestedColumnPtr();
|
||||
if (nullable_type)
|
||||
{
|
||||
auto casted_column = castRemoveNullable(nested, nullable_type->getNestedType());
|
||||
return ColumnNullable::create(casted_column, column_nullable->getNullMapColumnPtr());
|
||||
auto cast_column = castRemoveNullable(nested, nullable_type->getNestedType());
|
||||
return ColumnNullable::create(cast_column, column_nullable->getNullMapColumnPtr());
|
||||
}
|
||||
return castRemoveNullable(nested, data_type);
|
||||
}
|
||||
@ -229,8 +229,8 @@ ColumnPtr FunctionArrayIntersect<Mode>::castRemoveNullable(const ColumnPtr & col
|
||||
data_type->getName(),
|
||||
getName());
|
||||
|
||||
auto casted_column = castRemoveNullable(column_array->getDataPtr(), array_type->getNestedType());
|
||||
return ColumnArray::create(casted_column, column_array->getOffsetsPtr());
|
||||
auto cast_column = castRemoveNullable(column_array->getDataPtr(), array_type->getNestedType());
|
||||
return ColumnArray::create(cast_column, column_array->getOffsetsPtr());
|
||||
}
|
||||
if (const auto * column_tuple = checkAndGetColumn<ColumnTuple>(column.get()))
|
||||
{
|
||||
@ -261,7 +261,7 @@ FunctionArrayIntersect<Mode>::CastArgumentsResult FunctionArrayIntersect<Mode>::
|
||||
{
|
||||
size_t num_args = arguments.size();
|
||||
ColumnsWithTypeAndName initial_columns(num_args);
|
||||
ColumnsWithTypeAndName casted_columns(num_args);
|
||||
ColumnsWithTypeAndName cast_columns(num_args);
|
||||
|
||||
const auto * type_array = checkAndGetDataType<DataTypeArray>(return_type.get());
|
||||
const auto & type_nested = type_array->getNestedType();
|
||||
@ -288,8 +288,8 @@ FunctionArrayIntersect<Mode>::CastArgumentsResult FunctionArrayIntersect<Mode>::
|
||||
{
|
||||
const ColumnWithTypeAndName & arg = arguments[i];
|
||||
initial_columns[i] = arg;
|
||||
casted_columns[i] = arg;
|
||||
auto & column = casted_columns[i];
|
||||
cast_columns[i] = arg;
|
||||
auto & column = cast_columns[i];
|
||||
|
||||
if (is_numeric_or_string)
|
||||
{
|
||||
@ -332,7 +332,7 @@ FunctionArrayIntersect<Mode>::CastArgumentsResult FunctionArrayIntersect<Mode>::
|
||||
}
|
||||
}
|
||||
|
||||
return {.initial = initial_columns, .casted = casted_columns};
|
||||
return {.initial = initial_columns, .cast = cast_columns};
|
||||
}
|
||||
|
||||
static ColumnPtr callFunctionNotEquals(ColumnWithTypeAndName first, ColumnWithTypeAndName second, ContextPtr context)
|
||||
@ -385,7 +385,7 @@ FunctionArrayIntersect<Mode>::UnpackedArrays FunctionArrayIntersect<Mode>::prepa
|
||||
initial_column = &typeid_cast<const ColumnNullable &>(*initial_column).getNestedColumn();
|
||||
}
|
||||
|
||||
/// In case the column was casted, we need to create an overflow mask for integer types.
|
||||
/// In case the column was cast, we need to create an overflow mask for integer types.
|
||||
if (arg.nested_column != initial_column)
|
||||
{
|
||||
const auto & nested_init_type = typeid_cast<const DataTypeArray &>(*removeNullable(initial_columns[i].type)).getNestedType();
|
||||
@ -396,7 +396,7 @@ FunctionArrayIntersect<Mode>::UnpackedArrays FunctionArrayIntersect<Mode>::prepa
|
||||
|| isDateTime(nested_init_type)
|
||||
|| isDateTime64(nested_init_type))
|
||||
{
|
||||
/// Compare original and casted columns. It seem to be the easiest way.
|
||||
/// Compare original and cast columns. It seem to be the easiest way.
|
||||
auto overflow_mask = callFunctionNotEquals(
|
||||
{arg.nested_column->getPtr(), nested_init_type, ""},
|
||||
{initial_column->getPtr(), nested_cast_type, ""},
|
||||
@ -458,9 +458,9 @@ ColumnPtr FunctionArrayIntersect<Mode>::executeImpl(const ColumnsWithTypeAndName
|
||||
else
|
||||
return_type_with_nulls = getLeastSupertype(data_types);
|
||||
|
||||
auto casted_columns = castColumns(arguments, result_type, return_type_with_nulls);
|
||||
auto cast_columns = castColumns(arguments, result_type, return_type_with_nulls);
|
||||
|
||||
UnpackedArrays arrays = prepareArrays(casted_columns.casted, casted_columns.initial);
|
||||
UnpackedArrays arrays = prepareArrays(cast_columns.cast, cast_columns.initial);
|
||||
|
||||
ColumnPtr result_column;
|
||||
auto not_nullable_nested_return_type = removeNullable(nested_return_type);
|
||||
|
@ -336,7 +336,7 @@ The default value must be of the same type as the array elements.
|
||||
[example:simple_int_with_default]
|
||||
[example:simple_string_with_default]
|
||||
[example:simple_array_with_default]
|
||||
[example:casted_array_with_default]
|
||||
[example:cast_array_with_default]
|
||||
)",
|
||||
.examples{
|
||||
{"simple_int", "SELECT arrayShiftLeft([1, 2, 3, 4, 5], 3)", "[4, 5, 0, 0, 0]"},
|
||||
@ -348,7 +348,7 @@ The default value must be of the same type as the array elements.
|
||||
{"simple_int_with_default", "SELECT arrayShiftLeft([1, 2, 3, 4, 5], 3, 7)", "[4, 5, 7, 7, 7]"},
|
||||
{"simple_string_with_default", "SELECT arrayShiftLeft(['a', 'b', 'c', 'd', 'e'], 3, 'foo')", "['d', 'e', 'foo', 'foo', 'foo']"},
|
||||
{"simple_array_with_default", "SELECT arrayShiftLeft([[1, 2], [3, 4], [5, 6]], 2, [7, 8])", "[[5, 6], [7, 8], [7, 8]]"},
|
||||
{"casted_array_with_default",
|
||||
{"cast_array_with_default",
|
||||
"SELECT arrayShiftLeft(CAST('[1, 2, 3, 4, 5, 6]', 'Array(UInt16)'), 1, 1000)",
|
||||
"[2, 3, 4, 5, 6, 1000]"},
|
||||
},
|
||||
@ -374,7 +374,7 @@ The default value must be of the same type as the array elements.
|
||||
[example:simple_int_with_default]
|
||||
[example:simple_string_with_default]
|
||||
[example:simple_array_with_default]
|
||||
[example:casted_array_with_default]
|
||||
[example:cast_array_with_default]
|
||||
)",
|
||||
.examples{
|
||||
{"simple_int", "SELECT arrayShiftRight([1, 2, 3, 4, 5], 3)", "[0, 0, 0, 1, 2]"},
|
||||
@ -388,7 +388,7 @@ The default value must be of the same type as the array elements.
|
||||
"SELECT arrayShiftRight(['a', 'b', 'c', 'd', 'e'], 3, 'foo')",
|
||||
"['foo', 'foo', 'foo', 'a', 'b']"},
|
||||
{"simple_array_with_default", "SELECT arrayShiftRight([[1, 2], [3, 4], [5, 6]], 2, [7, 8])", "[[7, 8], [7, 8], [1, 2]]"},
|
||||
{"casted_array_with_default",
|
||||
{"cast_array_with_default",
|
||||
"SELECT arrayShiftRight(CAST('[1, 2, 3, 4, 5, 6]', 'Array(UInt16)'), 1, 1000)",
|
||||
"[1000, 1, 2, 3, 4, 5]"},
|
||||
},
|
||||
|
@ -108,29 +108,29 @@ public:
|
||||
const ColumnWithTypeAndName & offset_elem = arguments[1];
|
||||
bool has_defaults = arguments.size() == 3;
|
||||
|
||||
ColumnPtr source_column_casted = castColumn(source_elem, result_type);
|
||||
ColumnPtr source_column_cast = castColumn(source_elem, result_type);
|
||||
ColumnPtr offset_column = offset_elem.column;
|
||||
|
||||
ColumnPtr default_column_casted;
|
||||
ColumnPtr default_column_cast;
|
||||
if (has_defaults)
|
||||
{
|
||||
const ColumnWithTypeAndName & default_elem = arguments[2];
|
||||
default_column_casted = castColumn(default_elem, result_type);
|
||||
default_column_cast = castColumn(default_elem, result_type);
|
||||
}
|
||||
|
||||
bool source_is_constant = isColumnConst(*source_column_casted);
|
||||
bool source_is_constant = isColumnConst(*source_column_cast);
|
||||
bool offset_is_constant = isColumnConst(*offset_column);
|
||||
|
||||
bool default_is_constant = false;
|
||||
if (has_defaults)
|
||||
default_is_constant = isColumnConst(*default_column_casted);
|
||||
default_is_constant = isColumnConst(*default_column_cast);
|
||||
|
||||
if (source_is_constant)
|
||||
source_column_casted = assert_cast<const ColumnConst &>(*source_column_casted).getDataColumnPtr();
|
||||
source_column_cast = assert_cast<const ColumnConst &>(*source_column_cast).getDataColumnPtr();
|
||||
if (offset_is_constant)
|
||||
offset_column = assert_cast<const ColumnConst &>(*offset_column).getDataColumnPtr();
|
||||
if (default_is_constant)
|
||||
default_column_casted = assert_cast<const ColumnConst &>(*default_column_casted).getDataColumnPtr();
|
||||
default_column_cast = assert_cast<const ColumnConst &>(*default_column_cast).getDataColumnPtr();
|
||||
|
||||
if (offset_is_constant)
|
||||
{
|
||||
@ -177,18 +177,18 @@ public:
|
||||
{
|
||||
/// Degenerate case, just copy source column as is.
|
||||
return source_is_constant
|
||||
? ColumnConst::create(source_column_casted, input_rows_count)
|
||||
: source_column_casted;
|
||||
? ColumnConst::create(source_column_cast, input_rows_count)
|
||||
: source_column_cast;
|
||||
}
|
||||
if (offset > 0)
|
||||
{
|
||||
insert_range_from(source_is_constant, source_column_casted, offset, static_cast<Int64>(input_rows_count) - offset);
|
||||
insert_range_from(default_is_constant, default_column_casted, static_cast<Int64>(input_rows_count) - offset, offset);
|
||||
insert_range_from(source_is_constant, source_column_cast, offset, static_cast<Int64>(input_rows_count) - offset);
|
||||
insert_range_from(default_is_constant, default_column_cast, static_cast<Int64>(input_rows_count) - offset, offset);
|
||||
return result_column;
|
||||
}
|
||||
|
||||
insert_range_from(default_is_constant, default_column_casted, 0, -offset);
|
||||
insert_range_from(source_is_constant, source_column_casted, 0, static_cast<Int64>(input_rows_count) + offset);
|
||||
insert_range_from(default_is_constant, default_column_cast, 0, -offset);
|
||||
insert_range_from(source_is_constant, source_column_cast, 0, static_cast<Int64>(input_rows_count) + offset);
|
||||
return result_column;
|
||||
}
|
||||
|
||||
@ -205,9 +205,9 @@ public:
|
||||
Int64 src_idx = row + offset;
|
||||
|
||||
if (src_idx >= 0 && src_idx < static_cast<Int64>(input_rows_count))
|
||||
result_column->insertFrom(*source_column_casted, source_is_constant ? 0 : src_idx);
|
||||
result_column->insertFrom(*source_column_cast, source_is_constant ? 0 : src_idx);
|
||||
else if (has_defaults)
|
||||
result_column->insertFrom(*default_column_casted, default_is_constant ? 0 : row);
|
||||
result_column->insertFrom(*default_column_cast, default_is_constant ? 0 : row);
|
||||
else
|
||||
result_column->insertDefault();
|
||||
}
|
||||
|
@ -678,8 +678,8 @@ namespace
|
||||
}
|
||||
else
|
||||
{
|
||||
const auto * result_type_without_nullable_casted = checkAndGetDataType<DataTypeDateTime64>(result_type_without_nullable.get());
|
||||
MutableColumnPtr col_res = ColumnDateTime64::create(input_rows_count, result_type_without_nullable_casted->getScale());
|
||||
const auto * result_type_without_nullable_cast = checkAndGetDataType<DataTypeDateTime64>(result_type_without_nullable.get());
|
||||
MutableColumnPtr col_res = ColumnDateTime64::create(input_rows_count, result_type_without_nullable_cast->getScale());
|
||||
ColumnDateTime64 * col_datetime64 = assert_cast<ColumnDateTime64 *>(col_res.get());
|
||||
return executeImpl2<DataTypeDateTime64::FieldType>(arguments, result_type, input_rows_count, col_res, col_datetime64->getData());
|
||||
}
|
||||
@ -701,8 +701,8 @@ namespace
|
||||
UInt32 scale = 0;
|
||||
if constexpr (return_type == ReturnType::DateTime64)
|
||||
{
|
||||
const DataTypeDateTime64 * result_type_without_nullable_casted = checkAndGetDataType<DataTypeDateTime64>(removeNullable(result_type).get());
|
||||
scale = result_type_without_nullable_casted->getScale();
|
||||
const DataTypeDateTime64 * result_type_without_nullable_cast = checkAndGetDataType<DataTypeDateTime64>(removeNullable(result_type).get());
|
||||
scale = result_type_without_nullable_cast->getScale();
|
||||
multiplier = DecimalUtils::scaleMultiplier<DateTime64>(scale);
|
||||
}
|
||||
|
||||
|
@ -159,9 +159,9 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
ColumnPtr in_casted = arguments[0].column;
|
||||
ColumnPtr in_cast = arguments[0].column;
|
||||
if (arguments.size() == 3)
|
||||
in_casted = castColumn(arguments[0], result_type);
|
||||
in_cast = castColumn(arguments[0], result_type);
|
||||
|
||||
auto column_result = result_type->createColumn();
|
||||
if (cache.is_empty)
|
||||
@ -172,18 +172,18 @@ namespace
|
||||
}
|
||||
if (cache.table_num_to_idx)
|
||||
{
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const, *in_casted, input_rows_count)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const, *in_casted, input_rows_count))
|
||||
if (!executeNum<ColumnVector<UInt8>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt16>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt32>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<UInt64>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int8>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int16>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int32>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Int64>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Float32>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnVector<Float64>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnDecimal<Decimal32>>(in, *column_result, default_non_const, *in_cast, input_rows_count)
|
||||
&& !executeNum<ColumnDecimal<Decimal64>>(in, *column_result, default_non_const, *in_cast, input_rows_count))
|
||||
{
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of first argument of function {}", in->getName(), getName());
|
||||
@ -191,12 +191,12 @@ namespace
|
||||
}
|
||||
else if (cache.table_string_to_idx)
|
||||
{
|
||||
if (!executeString(in, *column_result, default_non_const, *in_casted, input_rows_count))
|
||||
executeContiguous(in, *column_result, default_non_const, *in_casted, input_rows_count);
|
||||
if (!executeString(in, *column_result, default_non_const, *in_cast, input_rows_count))
|
||||
executeContiguous(in, *column_result, default_non_const, *in_cast, input_rows_count);
|
||||
}
|
||||
else if (cache.table_anything_to_idx)
|
||||
{
|
||||
executeAnything(in, *column_result, default_non_const, *in_casted, input_rows_count);
|
||||
executeAnything(in, *column_result, default_non_const, *in_cast, input_rows_count);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "State of the function `transform` is not initialized");
|
||||
@ -217,7 +217,7 @@ namespace
|
||||
return impl->execute(args, result_type, input_rows_count, /* dry_run = */ false);
|
||||
}
|
||||
|
||||
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
|
||||
void executeAnything(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_cast, size_t input_rows_count) const
|
||||
{
|
||||
const auto & table = *cache.table_anything_to_idx;
|
||||
column_result.reserve(input_rows_count);
|
||||
@ -234,11 +234,11 @@ namespace
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(in_casted, i);
|
||||
column_result.insertFrom(in_cast, i);
|
||||
}
|
||||
}
|
||||
|
||||
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
|
||||
void executeContiguous(const IColumn * in, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_cast, size_t input_rows_count) const
|
||||
{
|
||||
const auto & table = *cache.table_string_to_idx;
|
||||
column_result.reserve(input_rows_count);
|
||||
@ -252,12 +252,12 @@ namespace
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(in_casted, i);
|
||||
column_result.insertFrom(in_cast, i);
|
||||
}
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
|
||||
bool executeNum(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_cast, size_t input_rows_count) const
|
||||
{
|
||||
const auto * const in = checkAndGetColumn<T>(in_untyped);
|
||||
if (!in)
|
||||
@ -293,7 +293,7 @@ namespace
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(in_casted, i);
|
||||
column_result.insertFrom(in_cast, i);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
@ -445,7 +445,7 @@ namespace
|
||||
}
|
||||
}
|
||||
|
||||
bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_casted, size_t input_rows_count) const
|
||||
bool executeString(const IColumn * in_untyped, IColumn & column_result, const ColumnPtr default_non_const, const IColumn & in_cast, size_t input_rows_count) const
|
||||
{
|
||||
const auto * const in = checkAndGetColumn<ColumnString>(in_untyped);
|
||||
if (!in)
|
||||
@ -482,7 +482,7 @@ namespace
|
||||
else if (default_non_const)
|
||||
column_result.insertFrom(*default_non_const, i);
|
||||
else
|
||||
column_result.insertFrom(in_casted, i);
|
||||
column_result.insertFrom(in_cast, i);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
@ -700,11 +700,11 @@ namespace
|
||||
throw Exception(
|
||||
ErrorCodes::ILLEGAL_COLUMN, "Second and third arguments of function {} must be constant arrays.", getName());
|
||||
|
||||
const ColumnPtr & from_column_uncasted = array_from->getDataPtr();
|
||||
const ColumnPtr & from_column_uncast = array_from->getDataPtr();
|
||||
|
||||
cache.from_column = castColumn(
|
||||
{
|
||||
from_column_uncasted,
|
||||
from_column_uncast,
|
||||
typeid_cast<const DataTypeArray &>(*arguments[1].type).getNestedType(),
|
||||
arguments[1].name
|
||||
},
|
||||
@ -757,7 +757,7 @@ namespace
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (which.isEnum() /// The correctness of strings are already checked by casting them to the Enum type.
|
||||
|| applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
|| applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncast)[i]))
|
||||
{
|
||||
UInt64 key = 0;
|
||||
auto * dst = reinterpret_cast<char *>(&key);
|
||||
@ -780,7 +780,7 @@ namespace
|
||||
auto & table = *cache.table_string_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncast)[i]))
|
||||
{
|
||||
StringRef ref = cache.from_column->getDataAt(i);
|
||||
table.insertIfNotPresent(ref, i);
|
||||
@ -793,7 +793,7 @@ namespace
|
||||
auto & table = *cache.table_anything_to_idx;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncasted)[i]))
|
||||
if (applyVisitor(FieldVisitorAccurateEquals(), (*cache.from_column)[i], (*from_column_uncast)[i]))
|
||||
{
|
||||
SipHash hash;
|
||||
cache.from_column->updateHashWithValue(i, hash);
|
||||
|
@ -130,21 +130,21 @@ class FunctionWidthBucket : public IFunction
|
||||
using ResultType = typename NumberTraits::Construct<false, false, NumberTraits::nextSize(sizeof(TCountType))>::Type;
|
||||
auto common_type = std::make_shared<DataTypeNumber<Float64>>();
|
||||
|
||||
std::vector<ColumnPtr> casted_columns;
|
||||
casted_columns.reserve(3);
|
||||
std::vector<ColumnPtr> cast_columns;
|
||||
cast_columns.reserve(3);
|
||||
for (const auto argument_index : collections::range(0, 3))
|
||||
{
|
||||
casted_columns.push_back(castColumn(arguments[argument_index], common_type));
|
||||
cast_columns.push_back(castColumn(arguments[argument_index], common_type));
|
||||
}
|
||||
|
||||
const auto * operands_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[0].get()));
|
||||
const auto * lows_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[1].get()));
|
||||
const auto * highs_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(casted_columns[2].get()));
|
||||
const auto * operands_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(cast_columns[0].get()));
|
||||
const auto * lows_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(cast_columns[1].get()));
|
||||
const auto * highs_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<Float64>>(cast_columns[2].get()));
|
||||
const auto * counts_vec = getDataIfNotNull(checkAndGetColumn<ColumnVector<TCountType>>(arguments[3].column.get()));
|
||||
|
||||
const auto * operands_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[0].get());
|
||||
const auto * lows_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[1].get());
|
||||
const auto * highs_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(casted_columns[2].get());
|
||||
const auto * operands_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(cast_columns[0].get());
|
||||
const auto * lows_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(cast_columns[1].get());
|
||||
const auto * highs_col_const = checkAndGetColumnConst<ColumnVector<Float64>>(cast_columns[2].get());
|
||||
const auto * counts_col_const = checkAndGetColumnConst<ColumnVector<TCountType>>(arguments[3].column.get());
|
||||
|
||||
throwIfInvalid<Float64>(0, operands_col_const, operands_vec, input_rows_count);
|
||||
|
@ -1459,7 +1459,7 @@ ActionsDAG ActionsDAG::makeConvertingActions(
|
||||
const ColumnsWithTypeAndName & result,
|
||||
MatchColumnsMode mode,
|
||||
bool ignore_constant_values,
|
||||
bool add_casted_columns,
|
||||
bool add_cast_columns,
|
||||
NameToNameMap * new_names)
|
||||
{
|
||||
size_t num_input_columns = source.size();
|
||||
@ -1468,8 +1468,8 @@ ActionsDAG ActionsDAG::makeConvertingActions(
|
||||
if (mode == MatchColumnsMode::Position && num_input_columns != num_result_columns)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH, "Number of columns doesn't match (source: {} and result: {})", num_input_columns, num_result_columns);
|
||||
|
||||
if (add_casted_columns && mode != MatchColumnsMode::Name)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_casted_columns supported only for MatchColumnsMode::Name");
|
||||
if (add_cast_columns && mode != MatchColumnsMode::Name)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Converting with add_cast_columns supported only for MatchColumnsMode::Name");
|
||||
|
||||
ActionsDAG actions_dag(source);
|
||||
NodeRawConstPtrs projection(num_result_columns);
|
||||
@ -1567,7 +1567,7 @@ ActionsDAG ActionsDAG::makeConvertingActions(
|
||||
|
||||
if (dst_node->result_name != res_elem.name)
|
||||
{
|
||||
if (add_casted_columns)
|
||||
if (add_cast_columns)
|
||||
{
|
||||
if (inputs.contains(dst_node->result_name))
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Cannot convert column `{}` to `{}` because other column have same name",
|
||||
|
@ -298,14 +298,14 @@ public:
|
||||
/// It is needed to convert result from different sources to the same structure, e.g. for UNION query.
|
||||
/// Conversion should be possible with only usage of CAST function and renames.
|
||||
/// @param ignore_constant_values - Do not check that constants are same. Use value from result_header.
|
||||
/// @param add_casted_columns - Create new columns with converted values instead of replacing original.
|
||||
/// @param new_names - Output parameter for new column names when add_casted_columns is used.
|
||||
/// @param add_cast_columns - Create new columns with converted values instead of replacing original.
|
||||
/// @param new_names - Output parameter for new column names when add_cast_columns is used.
|
||||
static ActionsDAG makeConvertingActions(
|
||||
const ColumnsWithTypeAndName & source,
|
||||
const ColumnsWithTypeAndName & result,
|
||||
MatchColumnsMode mode,
|
||||
bool ignore_constant_values = false,
|
||||
bool add_casted_columns = false,
|
||||
bool add_cast_columns = false,
|
||||
NameToNameMap * new_names = nullptr);
|
||||
|
||||
/// Create expression which add const column and then materialize it.
|
||||
|
@ -26,7 +26,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
/// Visitors consist of functions with unified interface 'void visit(Casted & x, ASTPtr & y)', there x is y, successfully casted to Casted.
|
||||
/// Visitors consist of functions with unified interface 'void visit(Cast & x, ASTPtr & y)', there x is y, successfully cast to Cast.
|
||||
/// Both types and function could have const specifiers. The second argument is used by visitor to replaces AST node (y) if needed.
|
||||
|
||||
/// Visits AST nodes, add default database to tables if not set. There's different logic for DDLs and selects.
|
||||
|
@ -744,7 +744,7 @@ static std::optional<ActionsDAG> changeKeyTypes(const ColumnsWithTypeAndName & c
|
||||
/* result= */ cols_dst,
|
||||
/* mode= */ ActionsDAG::MatchColumnsMode::Name,
|
||||
/* ignore_constant_values= */ true,
|
||||
/* add_casted_columns= */ add_new_cols,
|
||||
/* add_cast_columns= */ add_new_cols,
|
||||
/* new_names= */ &key_column_rename);
|
||||
}
|
||||
|
||||
@ -771,7 +771,7 @@ static std::optional<ActionsDAG> changeTypesToNullable(
|
||||
/* result= */ cols_dst,
|
||||
/* mode= */ ActionsDAG::MatchColumnsMode::Name,
|
||||
/* ignore_constant_values= */ true,
|
||||
/* add_casted_columns= */ false,
|
||||
/* add_cast_columns= */ false,
|
||||
/* new_names= */ nullptr);
|
||||
}
|
||||
|
||||
|
@ -597,15 +597,15 @@ namespace
|
||||
if (column->size() > max_elements)
|
||||
return {};
|
||||
|
||||
ColumnPtr casted_col;
|
||||
ColumnPtr cast_col;
|
||||
const NullMap * null_map = nullptr;
|
||||
|
||||
if (!type->equals(*node->result_type))
|
||||
{
|
||||
casted_col = tryCastColumn(column, value->result_type, node->result_type);
|
||||
if (!casted_col)
|
||||
cast_col = tryCastColumn(column, value->result_type, node->result_type);
|
||||
if (!cast_col)
|
||||
return {};
|
||||
const auto & col_nullable = assert_cast<const ColumnNullable &>(*casted_col);
|
||||
const auto & col_nullable = assert_cast<const ColumnNullable &>(*cast_col);
|
||||
null_map = &col_nullable.getNullMapData();
|
||||
column = col_nullable.getNestedColumnPtr();
|
||||
}
|
||||
|
@ -42,8 +42,8 @@ ColumnPtr tryGetSubcolumnFromBlock(const Block & block, const DataTypePtr & requ
|
||||
/// extract the subcolumn, because the data of dynamic subcolumn can change after cast.
|
||||
if ((elem->type->hasDynamicSubcolumns() || requested_column_type->hasDynamicSubcolumns()) && !elem->type->equals(*requested_column_type))
|
||||
{
|
||||
auto casted_column = castColumn({elem->column, elem->type, ""}, requested_column_type);
|
||||
auto elem_column = requested_column_type->tryGetSubcolumn(subcolumn_name, casted_column);
|
||||
auto cast_column = castColumn({elem->column, elem->type, ""}, requested_column_type);
|
||||
auto elem_column = requested_column_type->tryGetSubcolumn(subcolumn_name, cast_column);
|
||||
auto elem_type = requested_column_type->tryGetSubcolumnType(subcolumn_name);
|
||||
|
||||
if (!elem_type || !elem_column)
|
||||
|
@ -44,7 +44,7 @@ private:
|
||||
void tryLogSplit(const Poco::Message & msg);
|
||||
|
||||
using ChannelPtr = Poco::AutoPtr<Poco::Channel>;
|
||||
/// Handler and its pointer casted to extended interface
|
||||
/// Handler and its pointer cast to extended interface
|
||||
using ExtendedChannelPtrPair = std::pair<ChannelPtr, ExtendedLogChannel *>;
|
||||
std::map<std::string, ExtendedChannelPtrPair> channels;
|
||||
|
||||
|
@ -120,12 +120,12 @@ public:
|
||||
if (!child)
|
||||
return;
|
||||
|
||||
T * casted = dynamic_cast<T *>(child.get());
|
||||
if (!casted)
|
||||
T * cast = dynamic_cast<T *>(child.get());
|
||||
if (!cast)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not cast AST subtree");
|
||||
|
||||
children.push_back(child);
|
||||
field = casted;
|
||||
field = cast;
|
||||
}
|
||||
|
||||
template <typename T>
|
||||
@ -134,8 +134,8 @@ public:
|
||||
if (!child)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trying to replace AST subtree with nullptr");
|
||||
|
||||
T * casted = dynamic_cast<T *>(child.get());
|
||||
if (!casted)
|
||||
T * cast = dynamic_cast<T *>(child.get());
|
||||
if (!cast)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Could not cast AST subtree");
|
||||
|
||||
for (ASTPtr & current_child : children)
|
||||
@ -143,7 +143,7 @@ public:
|
||||
if (current_child.get() == field)
|
||||
{
|
||||
current_child = child;
|
||||
field = casted;
|
||||
field = cast;
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
@ -1153,7 +1153,7 @@ void WindowTransform::appendChunk(Chunk & chunk)
|
||||
// Initialize output columns.
|
||||
for (auto & ws : workspaces)
|
||||
{
|
||||
block.casted_columns.push_back(ws.window_function_impl ? ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices) : nullptr);
|
||||
block.cast_columns.push_back(ws.window_function_impl ? ws.window_function_impl->castColumn(block.input_columns, ws.argument_column_indices) : nullptr);
|
||||
|
||||
block.output_columns.push_back(ws.aggregate_function->getResultType()
|
||||
->createColumn());
|
||||
@ -2401,8 +2401,8 @@ struct WindowFunctionLagLeadInFrame final : public StatelessWindowFunction
|
||||
{
|
||||
// Column with default values is specified.
|
||||
const IColumn & default_column =
|
||||
current_block.casted_columns[function_index] ?
|
||||
*current_block.casted_columns[function_index].get() :
|
||||
current_block.cast_columns[function_index] ?
|
||||
*current_block.cast_columns[function_index].get() :
|
||||
*current_block.input_columns[workspace.argument_column_indices[2]].get();
|
||||
|
||||
to.insert(default_column[transform->current_row.row]);
|
||||
|
@ -27,7 +27,7 @@ struct WindowTransformBlock
|
||||
{
|
||||
Columns original_input_columns;
|
||||
Columns input_columns;
|
||||
Columns casted_columns;
|
||||
Columns cast_columns;
|
||||
MutableColumns output_columns;
|
||||
|
||||
size_t rows = 0;
|
||||
|
@ -279,7 +279,7 @@ struct DefaultExpressionsInfo
|
||||
void getDefaultExpressionInfoInto(const ASTColumnDeclaration & col_decl, const DataTypePtr & data_type, DefaultExpressionsInfo & info);
|
||||
|
||||
/// Validate default expressions and corresponding types compatibility, i.e.
|
||||
/// default expression result can be casted to column_type. Also checks, that we
|
||||
/// default expression result can be cast to column_type. Also checks, that we
|
||||
/// don't have strange constructions in default expression like SELECT query or
|
||||
/// arrayJoin function.
|
||||
void validateColumnsDefaults(ASTPtr default_expr_list, const NamesAndTypesList & all_columns, ContextPtr context);
|
||||
|
@ -980,13 +980,13 @@ bool applyFunctionChainToColumn(
|
||||
|
||||
// And cast it to the argument type of the first function in the chain
|
||||
auto in_argument_type = getArgumentTypeOfMonotonicFunction(*functions[0]);
|
||||
if (canBeSafelyCasted(result_type, in_argument_type))
|
||||
if (canBeSafelyCast(result_type, in_argument_type))
|
||||
{
|
||||
result_column = castColumnAccurate({result_column, result_type, ""}, in_argument_type);
|
||||
result_type = in_argument_type;
|
||||
}
|
||||
// If column cannot be casted accurate, casting with OrNull, and in case all
|
||||
// values has been casted (no nulls), unpacking nested column from nullable.
|
||||
// If column cannot be cast accurate, casting with OrNull, and in case all
|
||||
// values has been cast (no nulls), unpacking nested column from nullable.
|
||||
// In case any further functions require Nullable input, they'll be able
|
||||
// to cast it.
|
||||
else
|
||||
@ -1009,7 +1009,7 @@ bool applyFunctionChainToColumn(
|
||||
return false;
|
||||
|
||||
auto argument_type = getArgumentTypeOfMonotonicFunction(*func);
|
||||
if (!canBeSafelyCasted(result_type, argument_type))
|
||||
if (!canBeSafelyCast(result_type, argument_type))
|
||||
return false;
|
||||
|
||||
result_column = castColumnAccurate({result_column, result_type, ""}, argument_type);
|
||||
@ -1309,7 +1309,7 @@ bool KeyCondition::tryPrepareSetIndex(
|
||||
is_constant_transformed = true;
|
||||
}
|
||||
|
||||
if (canBeSafelyCasted(set_element_type, key_column_type))
|
||||
if (canBeSafelyCast(set_element_type, key_column_type))
|
||||
{
|
||||
transformed_set_columns[set_element_index] = castColumn({set_column, set_element_type, {}}, key_column_type);
|
||||
continue;
|
||||
@ -2100,7 +2100,7 @@ bool KeyCondition::extractAtomFromTree(const RPNBuilderTreeNode & node, RPNEleme
|
||||
else
|
||||
key_expr_type_not_null = key_expr_type;
|
||||
|
||||
bool cast_not_needed = is_set_const /// Set args are already casted inside Set::createFromAST
|
||||
bool cast_not_needed = is_set_const /// Set args are already cast inside Set::createFromAST
|
||||
|| ((isNativeInteger(key_expr_type_not_null) || isDateTime(key_expr_type_not_null))
|
||||
&& (isNativeInteger(const_type) || isDateTime(const_type))); /// Native integers and DateTime are accurately compared without cast.
|
||||
|
||||
|
@ -151,7 +151,7 @@ const ActionsDAG::Node & addFunction(
|
||||
|
||||
/// Adds a CAST node with the regular name ("CAST(...)") or with the provided name.
|
||||
/// This is different from ActionsDAG::addCast() because it set the name equal to the original name effectively hiding the value before cast,
|
||||
/// but it might be required for further steps with its original uncasted type.
|
||||
/// but it might be required for further steps with its original uncast type.
|
||||
const ActionsDAG::Node & addCast(
|
||||
const ActionsDAGPtr & dag,
|
||||
const ActionsDAG::Node & node_to_cast,
|
||||
@ -247,7 +247,7 @@ bool tryBuildPrewhereSteps(
|
||||
struct Step
|
||||
{
|
||||
ActionsDAGPtr actions;
|
||||
/// Original condition, in case if we have only one condition, and it was not casted
|
||||
/// Original condition, in case if we have only one condition, and it was not cast
|
||||
const ActionsDAG::Node * original_node;
|
||||
/// Result condition node
|
||||
const ActionsDAG::Node * result_node;
|
||||
|
@ -10,8 +10,8 @@
|
||||
[90m2. [0m│ \ │ 0 │
|
||||
[90m3. [0m│ \t │ 0 │
|
||||
└──────────┴───┘
|
||||
┌─[1mx[0m────────┬─[1my[0m─┬─[1mtoInt8(x)[0m─┬─[1ms[0m─────┬─[1mcasted[0m─┐
|
||||
[90m1. [0m│ Hello │ 0 │ -100 │ Hello │ Hello │
|
||||
[90m2. [0m│ \ │ 0 │ 0 │ \ │ \ │
|
||||
[90m3. [0m│ \t │ 0 │ 111 │ \t │ \t │
|
||||
└──────────┴───┴───────────┴───────┴────────┘
|
||||
┌─[1mx[0m────────┬─[1my[0m─┬─[1mtoInt8(x)[0m─┬─[1ms[0m─────┬─[1mcast[0m──┐
|
||||
[90m1. [0m│ Hello │ 0 │ -100 │ Hello │ Hello │
|
||||
[90m2. [0m│ \ │ 0 │ 0 │ \ │ \ │
|
||||
[90m3. [0m│ \t │ 0 │ 111 │ \t │ \t │
|
||||
└──────────┴───┴───────────┴───────┴───────┘
|
||||
|
@ -8,6 +8,6 @@ INSERT INTO enum (x) VALUES ('\\');
|
||||
SELECT * FROM enum ORDER BY x, y FORMAT PrettyCompact;
|
||||
INSERT INTO enum (x) VALUES ('\t\\t');
|
||||
SELECT * FROM enum ORDER BY x, y FORMAT PrettyCompact;
|
||||
SELECT x, y, toInt8(x), toString(x) AS s, CAST(s AS Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111)) AS casted FROM enum ORDER BY x, y FORMAT PrettyCompact;
|
||||
SELECT x, y, toInt8(x), toString(x) AS s, CAST(s AS Enum8('Hello' = -100, '\\' = 0, '\t\\t' = 111)) AS cast FROM enum ORDER BY x, y FORMAT PrettyCompact;
|
||||
|
||||
DROP TABLE enum;
|
||||
|
@ -61,12 +61,12 @@ Filter column: notEquals(__table1.y, 0_UInt8)
|
||||
7 8
|
||||
8 9
|
||||
9 10
|
||||
> one condition of filter should be pushed down after aggregating, other condition is casted
|
||||
> one condition of filter should be pushed down after aggregating, other condition is cast
|
||||
Filter column
|
||||
FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))
|
||||
Aggregating
|
||||
Filter column: notEquals(y, 0)
|
||||
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted
|
||||
> (analyzer) one condition of filter should be pushed down after aggregating, other condition is cast
|
||||
Filter column
|
||||
FUNCTION and(minus(__table1.s, 4_UInt8) :: 1, 1 :: 3) -> and(notEquals(__table1.y, 0_UInt8), minus(__table1.s, 4_UInt8))
|
||||
Aggregating
|
||||
|
@ -63,14 +63,14 @@ $CLICKHOUSE_CLIENT -q "
|
||||
) where y != 0 and s != 4 order by s, y
|
||||
settings enable_optimize_predicate_expression=0"
|
||||
|
||||
echo "> one condition of filter should be pushed down after aggregating, other condition is casted"
|
||||
echo "> one condition of filter should be pushed down after aggregating, other condition is cast"
|
||||
$CLICKHOUSE_CLIENT --enable_analyzer=0 -q "
|
||||
explain actions = 1 select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
) where y != 0 and s - 4
|
||||
settings enable_optimize_predicate_expression=0" |
|
||||
grep -o "Aggregating\|Filter column\|Filter column: notEquals(y, 0)\|FUNCTION and(minus(s, 4) :: 5, 1 :: 3) -> and(notEquals(y, 0), minus(s, 4))"
|
||||
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is casted"
|
||||
echo "> (analyzer) one condition of filter should be pushed down after aggregating, other condition is cast"
|
||||
$CLICKHOUSE_CLIENT --enable_analyzer=1 -q "
|
||||
explain actions = 1 select s, y from (
|
||||
select sum(x) as s, y from (select number as x, number + 1 as y from numbers(10)) group by y
|
||||
|
@ -1388,7 +1388,6 @@ bools
|
||||
boringssl
|
||||
boundingRatio
|
||||
bozerkins
|
||||
broadcasted
|
||||
brotli
|
||||
bson
|
||||
bsoneachrow
|
||||
@ -1414,7 +1413,6 @@ cardinalities
|
||||
cardinality
|
||||
cartesian
|
||||
cassandra
|
||||
casted
|
||||
catboost
|
||||
catboostEvaluate
|
||||
categoricalInformationValue
|
||||
|
Loading…
Reference in New Issue
Block a user