Merge branch 'master' into Avogar-patch-1

This commit is contained in:
Sergei Trifonov 2022-12-21 11:54:18 +01:00 committed by GitHub
commit 048525273f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 89 additions and 35 deletions

View File

@ -3447,13 +3447,18 @@ Default value: 2.
## compatibility {#compatibility} ## compatibility {#compatibility}
This setting changes other settings according to provided ClickHouse version. The `compatibility` setting causes ClickHouse to use the default settings of a previous version of ClickHouse, where the previous version is provided as the setting.
If a behaviour in ClickHouse was changed by using a different default value for some setting, this compatibility setting allows you to use default values from previous versions for all the settings that were not set by the user.
This setting takes ClickHouse version number as a string, like `21.3`, `21.8`. Empty value means that this setting is disabled. If settings are set to non-default values, then those settings are honored (only settings that have not been modified are affected by the `compatibility` setting).
This setting takes a ClickHouse version number as a string, like `22.3`, `22.8`. An empty value means that this setting is disabled.
Disabled by default. Disabled by default.
:::note
In ClickHouse Cloud the compatibility setting must be set by ClickHouse Cloud support. Please [open a case](https://clickhouse.cloud/support) to have it set.
:::
## allow_settings_after_format_in_insert {#allow_settings_after_format_in_insert} ## allow_settings_after_format_in_insert {#allow_settings_after_format_in_insert}
Control whether `SETTINGS` after `FORMAT` in `INSERT` queries is allowed or not. It is not recommended to use this, since this may interpret part of `SETTINGS` as values. Control whether `SETTINGS` after `FORMAT` in `INSERT` queries is allowed or not. It is not recommended to use this, since this may interpret part of `SETTINGS` as values.

View File

@ -7,6 +7,8 @@ namespace DB
{ {
static const uint8_t BSON_DOCUMENT_END = 0x00; static const uint8_t BSON_DOCUMENT_END = 0x00;
static const size_t BSON_OBJECT_ID_SIZE = 12;
static const size_t BSON_DB_POINTER_SIZE = 12;
using BSONSizeT = uint32_t; using BSONSizeT = uint32_t;
static const BSONSizeT MAX_BSON_SIZE = std::numeric_limits<BSONSizeT>::max(); static const BSONSizeT MAX_BSON_SIZE = std::numeric_limits<BSONSizeT>::max();

View File

@ -685,37 +685,27 @@ public:
} }
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>) else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime64>)
{ {
if (typeid_cast<const DataTypeDateTime64 *>(arguments[0].type.get())) static constexpr auto target_scale = std::invoke(
[]() -> std::optional<UInt32>
{
if constexpr (std::is_base_of_v<AddNanosecondsImpl, Transform>)
return 9;
else if constexpr (std::is_base_of_v<AddMicrosecondsImpl, Transform>)
return 6;
else if constexpr (std::is_base_of_v<AddMillisecondsImpl, Transform>)
return 3;
return {};
});
auto timezone = extractTimeZoneNameFromFunctionArguments(arguments, 2, 0);
if (const auto* datetime64_type = typeid_cast<const DataTypeDateTime64 *>(arguments[0].type.get()))
{ {
const auto & datetime64_type = assert_cast<const DataTypeDateTime64 &>(*arguments[0].type); const auto from_scale = datetime64_type->getScale();
return std::make_shared<DataTypeDateTime64>(std::max(from_scale, target_scale.value_or(from_scale)), std::move(timezone));
auto from_scale = datetime64_type.getScale();
auto scale = from_scale;
if (std::is_same_v<Transform, AddNanosecondsImpl>)
scale = 9;
else if (std::is_same_v<Transform, AddMicrosecondsImpl>)
scale = 6;
else if (std::is_same_v<Transform, AddMillisecondsImpl>)
scale = 3;
scale = std::max(scale, from_scale);
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
} }
else
{
auto scale = DataTypeDateTime64::default_scale;
if (std::is_same_v<Transform, AddNanosecondsImpl>) return std::make_shared<DataTypeDateTime64>(target_scale.value_or(DataTypeDateTime64::default_scale), std::move(timezone));
scale = 9;
else if (std::is_same_v<Transform, AddMicrosecondsImpl>)
scale = 6;
else if (std::is_same_v<Transform, AddMillisecondsImpl>)
scale = 3;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 0));
}
} }
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type in datetime add interval function"); throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected result type in datetime add interval function");

View File

@ -18,6 +18,7 @@
#include <Columns/ColumnMap.h> #include <Columns/ColumnMap.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeUUID.h> #include <DataTypes/DataTypeUUID.h>
#include <DataTypes/DataTypeDateTime64.h> #include <DataTypes/DataTypeDateTime64.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
@ -282,7 +283,7 @@ static void readAndInsertString(ReadBuffer & in, IColumn & column, BSONType bson
} }
else if (bson_type == BSONType::OBJECT_ID) else if (bson_type == BSONType::OBJECT_ID)
{ {
readAndInsertStringImpl<is_fixed_string>(in, column, 12); readAndInsertStringImpl<is_fixed_string>(in, column, BSON_OBJECT_ID_SIZE);
} }
else else
{ {
@ -664,7 +665,7 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
} }
case BSONType::OBJECT_ID: case BSONType::OBJECT_ID:
{ {
in.ignore(12); in.ignore(BSON_OBJECT_ID_SIZE);
break; break;
} }
case BSONType::REGEXP: case BSONType::REGEXP:
@ -677,7 +678,7 @@ static void skipBSONField(ReadBuffer & in, BSONType type)
{ {
BSONSizeT size; BSONSizeT size;
readBinary(size, in); readBinary(size, in);
in.ignore(size + 12); in.ignore(size + BSON_DB_POINTER_SIZE);
break; break;
} }
case BSONType::JAVA_SCRIPT_CODE_W_SCOPE: case BSONType::JAVA_SCRIPT_CODE_W_SCOPE:
@ -796,7 +797,6 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
} }
case BSONType::SYMBOL: [[fallthrough]]; case BSONType::SYMBOL: [[fallthrough]];
case BSONType::JAVA_SCRIPT_CODE: [[fallthrough]]; case BSONType::JAVA_SCRIPT_CODE: [[fallthrough]];
case BSONType::OBJECT_ID: [[fallthrough]];
case BSONType::STRING: case BSONType::STRING:
{ {
BSONSizeT size; BSONSizeT size;
@ -804,6 +804,11 @@ DataTypePtr BSONEachRowSchemaReader::getDataTypeFromBSONField(BSONType type, boo
in.ignore(size); in.ignore(size);
return std::make_shared<DataTypeString>(); return std::make_shared<DataTypeString>();
} }
case BSONType::OBJECT_ID:;
{
in.ignore(BSON_OBJECT_ID_SIZE);
return makeNullable(std::make_shared<DataTypeFixedString>(BSON_OBJECT_ID_SIZE));
}
case BSONType::DOCUMENT: case BSONType::DOCUMENT:
{ {
auto nested_names_and_types = getDataTypesFromBSONDocument(false); auto nested_names_and_types = getDataTypesFromBSONDocument(false);
@ -954,6 +959,7 @@ void registerInputFormatBSONEachRow(FormatFactory & factory)
"BSONEachRow", "BSONEachRow",
[](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings) [](ReadBuffer & buf, const Block & sample, IRowInputFormat::Params params, const FormatSettings & settings)
{ return std::make_shared<BSONEachRowRowInputFormat>(buf, sample, std::move(params), settings); }); { return std::make_shared<BSONEachRowRowInputFormat>(buf, sample, std::move(params), settings); });
factory.registerFileExtension("bson", "BSONEachRow");
} }
void registerFileSegmentationEngineBSONEachRow(FormatFactory & factory) void registerFileSegmentationEngineBSONEachRow(FormatFactory & factory)

View File

@ -60,3 +60,19 @@ test add[...]seconds()
2220-12-12 12:12:12.124 2220-12-12 12:12:12.124
2220-12-12 12:12:12.121 2220-12-12 12:12:12.121
2220-12-12 12:12:12.124456 2220-12-12 12:12:12.124456
test subtract[...]seconds()
- test nanoseconds
2022-12-31 23:59:59.999999999
2022-12-31 23:59:59.999999900
2023-01-01 00:00:00.000000001
2023-01-01 00:00:00.000000100
- test microseconds
2022-12-31 23:59:59.999999
2022-12-31 23:59:59.999900
2023-01-01 00:00:00.000001
2023-01-01 00:00:00.000100
- test milliseconds
2022-12-31 23:59:59.999
2022-12-31 23:59:59.900
2023-01-01 00:00:00.001
2023-01-01 00:00:00.100

View File

@ -92,3 +92,22 @@ select addMilliseconds(toDateTime64('1930-12-12 12:12:12.123456', 6), 1); -- Bel
select addMilliseconds(toDateTime64('2220-12-12 12:12:12.123', 3), 1); -- Above normal range, source scale matches result select addMilliseconds(toDateTime64('2220-12-12 12:12:12.123', 3), 1); -- Above normal range, source scale matches result
select addMilliseconds(toDateTime64('2220-12-12 12:12:12.12', 2), 1); -- Above normal range, source scale less than result select addMilliseconds(toDateTime64('2220-12-12 12:12:12.12', 2), 1); -- Above normal range, source scale less than result
select addMilliseconds(toDateTime64('2220-12-12 12:12:12.123456', 6), 1); -- Above normal range, source scale greater than result select addMilliseconds(toDateTime64('2220-12-12 12:12:12.123456', 6), 1); -- Above normal range, source scale greater than result
select 'test subtract[...]seconds()';
select '- test nanoseconds';
select subtractNanoseconds(toDateTime64('2023-01-01 00:00:00.0000000', 7, 'UTC'), 1);
select subtractNanoseconds(toDateTime64('2023-01-01 00:00:00.0000000', 7, 'UTC'), 100);
select subtractNanoseconds(toDateTime64('2023-01-01 00:00:00.0000000', 7, 'UTC'), -1);
select subtractNanoseconds(toDateTime64('2023-01-01 00:00:00.0000000', 7, 'UTC'), -100);
select '- test microseconds';
select subtractMicroseconds(toDateTime64('2023-01-01 00:00:00.0000', 4, 'UTC'), 1);
select subtractMicroseconds(toDateTime64('2023-01-01 00:00:00.0000', 4, 'UTC'), 100);
select subtractMicroseconds(toDateTime64('2023-01-01 00:00:00.0000', 4, 'UTC'), -1);
select subtractMicroseconds(toDateTime64('2023-01-01 00:00:00.0000', 4, 'UTC'), -100);
select '- test milliseconds';
select subtractMilliseconds(toDateTime64('2023-01-01 00:00:00.0', 1, 'UTC'), 1);
select subtractMilliseconds(toDateTime64('2023-01-01 00:00:00.0', 1, 'UTC'), 100);
select subtractMilliseconds(toDateTime64('2023-01-01 00:00:00.0', 1, 'UTC'), -1);
select subtractMilliseconds(toDateTime64('2023-01-01 00:00:00.0', 1, 'UTC'), -100);

View File

@ -0,0 +1,6 @@
_id Nullable(FixedString(12))
name Nullable(String)
email Nullable(String)
movie_id Nullable(FixedString(12))
text Nullable(String)
date Nullable(DateTime64(6, \'UTC\'))

View File

@ -0,0 +1,10 @@
#!/usr/bin/env bash
# Tags: no-fasttest
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL -q "desc file('$CURDIR/data_bson/comments.bson')"
$CLICKHOUSE_LOCAL -q "select _id from file('$CURDIR/data_bson/comments.bson') format Null"

Binary file not shown.