Skip to content

Commit 0fbfe24

Browse files
committed
Merge remote-tracking branch 'rschu1ze/master' into bump-azure-1.8
2 parents 815d7e6 + 7ceb623 commit 0fbfe24

File tree

12 files changed

+109
-28
lines changed

12 files changed

+109
-28
lines changed

contrib/openssl-cmake/CMakeLists.txt

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -100,8 +100,9 @@ if(ARCH_AMD64)
100100
set(OPENSSL_SYSTEM "macosx")
101101
else()
102102
macro(perl_generate_asm FILE_IN FILE_OUT)
103+
# Manually set $CC because the called Perl scripts require it.
103104
add_custom_command(OUTPUT ${FILE_OUT}
104-
COMMAND /usr/bin/env perl ${FILE_IN} ${FILE_OUT})
105+
COMMAND ${CMAKE_COMMAND} -E env "CC=${CMAKE_CXX_COMPILER}" /usr/bin/env perl ${FILE_IN} ${FILE_OUT})
105106
endmacro()
106107

107108
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/aes-x86_64.pl ${OPENSSL_BINARY_DIR}/crypto/aes/aes-x86_64.s)

docs/en/sql-reference/functions/date-time-functions.md

Lines changed: 3 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1836,6 +1836,9 @@ Alias: `dateTrunc`.
18361836
- `unit` — The type of interval to truncate the result. [String Literal](../syntax.md#syntax-string-literal).
18371837
Possible values:
18381838

1839+
- `nanosecond` - Compatible only with DateTime64
1840+
- `microsecond` - Compatible only with DateTime64
1841+
- `milisecond` - Compatible only with DateTime64
18391842
- `second`
18401843
- `minute`
18411844
- `hour`

src/AggregateFunctions/AggregateFunctionLargestTriangleThreeBuckets.cpp

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -40,12 +40,16 @@ struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float
4040
{
4141
void add(const Float64 xval, const Float64 yval, Arena * arena)
4242
{
43+
/// We need to ensure either both or neither coordinates are saved (StatisticalSample ignores NaNs)
44+
if (isNaN(xval) || isNaN(yval))
45+
return;
4346
this->addX(xval, arena);
4447
this->addY(yval, arena);
4548
}
4649

4750
void sort(Arena * arena)
4851
{
52+
chassert(this->x.size() == this->y.size());
4953
// sort the this->x and this->y in ascending order of this->x using index
5054
std::vector<size_t> index(this->x.size());
5155

src/Functions/UTCTimestampTransform.cpp

Lines changed: 22 additions & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -67,7 +67,7 @@ namespace
6767
return date_time_type;
6868
}
6969

70-
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
70+
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
7171
{
7272
if (arguments.size() != 2)
7373
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 2.", name);
@@ -77,37 +77,47 @@ namespace
7777
if (!time_zone_const_col)
7878
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of 2nd argument of function {}. Excepted const(String).", arg2.column->getName(), name);
7979
String time_zone_val = time_zone_const_col->getDataAt(0).toString();
80-
auto column = result_type->createColumn();
80+
const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
8181
if (WhichDataType(arg1.type).isDateTime())
8282
{
8383
const auto * date_time_col = checkAndGetColumn<ColumnDateTime>(arg1.column.get());
84-
for (size_t i = 0; i < date_time_col->size(); ++i)
84+
size_t col_size = date_time_col->size();
85+
using ColVecTo = DataTypeDateTime::ColumnType;
86+
typename ColVecTo::MutablePtr result_column = ColVecTo::create(col_size);
87+
typename ColVecTo::Container & result_data = result_column->getData();
88+
for (size_t i = 0; i < col_size; ++i)
8589
{
8690
UInt32 date_time_val = date_time_col->getElement(i);
87-
LocalDateTime date_time(date_time_val, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
88-
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
89-
column->insert(time_val);
91+
LocalDateTime date_time(date_time_val, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));
92+
time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val));
93+
result_data[i] = static_cast<UInt32>(time_val);
9094
}
95+
return result_column;
9196
}
9297
else if (WhichDataType(arg1.type).isDateTime64())
9398
{
9499
const auto * date_time_col = checkAndGetColumn<ColumnDateTime64>(arg1.column.get());
100+
size_t col_size = date_time_col->size();
95101
const DataTypeDateTime64 * date_time_type = static_cast<const DataTypeDateTime64 *>(arg1.type.get());
96-
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(date_time_type->getScale());
97-
for (size_t i = 0; i < date_time_col->size(); ++i)
102+
UInt32 col_scale = date_time_type->getScale();
103+
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(col_scale);
104+
using ColDecimalTo = DataTypeDateTime64::ColumnType;
105+
typename ColDecimalTo::MutablePtr result_column = ColDecimalTo::create(col_size, col_scale);
106+
typename ColDecimalTo::Container & result_data = result_column->getData();
107+
for (size_t i = 0; i < col_size; ++i)
98108
{
99109
DateTime64 date_time_val = date_time_col->getElement(i);
100110
Int64 seconds = date_time_val.value / scale_multiplier;
101111
Int64 micros = date_time_val.value % scale_multiplier;
102-
LocalDateTime date_time(seconds, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
103-
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
112+
LocalDateTime date_time(seconds, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));
113+
time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val));
104114
DateTime64 date_time_64(time_val * scale_multiplier + micros);
105-
column->insert(date_time_64);
115+
result_data[i] = date_time_64;
106116
}
117+
return result_column;
107118
}
108119
else
109120
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 1st argument can only be datetime/datatime64. ", name);
110-
return column;
111121
}
112122

113123
};

src/Functions/date_trunc.cpp

Lines changed: 30 additions & 11 deletions
Original file line numberDiff line numberDiff line change
@@ -40,7 +40,14 @@ class FunctionDateTrunc : public IFunction
4040
{
4141
/// The first argument is a constant string with the name of datepart.
4242

43-
auto result_type_is_date = false;
43+
enum ResultType
44+
{
45+
Date,
46+
DateTime,
47+
DateTime64,
48+
};
49+
ResultType result_type;
50+
4451
String datepart_param;
4552
auto check_first_argument = [&] {
4653
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
@@ -56,13 +63,14 @@ class FunctionDateTrunc : public IFunction
5663
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
5764
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
5865

59-
if (datepart_kind == IntervalKind::Kind::Nanosecond || datepart_kind == IntervalKind::Kind::Microsecond
60-
|| datepart_kind == IntervalKind::Kind::Millisecond)
61-
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't support {}", getName(), datepart_param);
62-
63-
result_type_is_date = (datepart_kind == IntervalKind::Kind::Year)
64-
|| (datepart_kind == IntervalKind::Kind::Quarter) || (datepart_kind == IntervalKind::Kind::Month)
65-
|| (datepart_kind == IntervalKind::Kind::Week);
66+
if ((datepart_kind == IntervalKind::Kind::Year) || (datepart_kind == IntervalKind::Kind::Quarter)
67+
|| (datepart_kind == IntervalKind::Kind::Month) || (datepart_kind == IntervalKind::Kind::Week))
68+
result_type = ResultType::Date;
69+
else if ((datepart_kind == IntervalKind::Kind::Day) || (datepart_kind == IntervalKind::Kind::Hour)
70+
|| (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))
71+
result_type = ResultType::DateTime;
72+
else
73+
result_type = ResultType::DateTime64;
6674
};
6775

6876
bool second_argument_is_date = false;
@@ -84,7 +92,7 @@ class FunctionDateTrunc : public IFunction
8492
"This argument is optional and must be a constant string with timezone name",
8593
arguments[2].type->getName(), getName());
8694

87-
if (second_argument_is_date && result_type_is_date)
95+
if (second_argument_is_date && result_type == ResultType::Date)
8896
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
8997
"The timezone argument of function {} with datepart '{}' "
9098
"is allowed only when the 2nd argument has the type DateTime",
@@ -109,10 +117,21 @@ class FunctionDateTrunc : public IFunction
109117
getName(), arguments.size());
110118
}
111119

112-
if (result_type_is_date)
120+
if (result_type == ResultType::Date)
113121
return std::make_shared<DataTypeDate>();
114-
else
122+
else if (result_type == ResultType::DateTime)
115123
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
124+
else
125+
{
126+
size_t scale;
127+
if (datepart_kind == IntervalKind::Kind::Millisecond)
128+
scale = 3;
129+
else if (datepart_kind == IntervalKind::Kind::Microsecond)
130+
scale = 6;
131+
else if (datepart_kind == IntervalKind::Kind::Nanosecond)
132+
scale = 9;
133+
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
134+
}
116135
}
117136

118137
bool useDefaultImplementationForConstants() const override { return true; }

src/Interpreters/TransactionLog.cpp

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,7 @@
77
#include <IO/ReadBufferFromString.h>
88
#include <Common/Exception.h>
99
#include <Common/ZooKeeper/KeeperException.h>
10+
#include <Common/threadPoolCallbackRunner.h>
1011
#include <Core/ServerUUID.h>
1112
#include <Common/logger_useful.h>
1213
#include <Common/noexcept_scope.h>
@@ -451,6 +452,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool
451452
CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN allocated_csn, scope_guard & state_guard) noexcept
452453
{
453454
LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global);
455+
auto blocker = CannotAllocateThreadFaultInjector::blockFaultInjections();
454456
chassert(!allocated_csn == txn->isReadOnly());
455457
if (allocated_csn)
456458
{

tests/performance/scripts/eqmed.sql

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -67,5 +67,4 @@ from
6767
select throwIf(uniq((test, query)) != 1) from table
6868
) check_single_query -- this subselect checks that there is only one query in the input table;
6969
-- written this way so that it is not optimized away (#10523)
70-
SETTINGS allow_experimental_analyzer = 0
7170
;
Lines changed: 14 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,14 @@
1+
<test>
2+
<create_query>CREATE TABLE test1(d DateTime) ENGINE Memory</create_query>
3+
<create_query>CREATE TABLE test2(d DateTime64) ENGINE Memory</create_query>
4+
<fill_query>INSERT INTO test1 SELECT toDateTime('2023-03-16 11:22:33') + number from numbers(10000000)</fill_query>
5+
<fill_query>INSERT INTO test2 SELECT toDateTime64('2023-03-16 11:22:33', 3) + number from numbers(10000000)</fill_query>
6+
7+
<query tag="ToUtcTimestampDateTime">select count(1) from test1 where to_utc_timestamp(d, 'Etc/GMT+1') > '1990-01-01 12:00:00' SETTINGS max_threads=1</query>
8+
<query tag="FromUtcTimestampDateTime">select count(1) from test1 where from_utc_timestamp(d, 'Etc/GMT+1') > '1990-01-01 12:00:00' SETTINGS max_threads=1</query>
9+
<query tag="ToUtcTimestampDateTime64">select count(1) from test2 where to_utc_timestamp(d, 'Etc/GMT+1') > '1990-01-01 12:00:00' SETTINGS max_threads=1</query>
10+
<query tag="FromUtcTimestampDateTime64">select count(1) from test2 where from_utc_timestamp(d, 'Etc/GMT+1') > '1990-01-01 12:00:00' SETTINGS max_threads=1</query>
11+
12+
<drop_query>DROP TABLE IF EXISTS test1</drop_query>
13+
<drop_query>DROP TABLE IF EXISTS test2</drop_query>
14+
</test>

tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.reference

Lines changed: 12 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -7,3 +7,15 @@
77
2022-03-01 00:00:00
88
2022-03-01
99
2022-02-28
10+
2022-03-01 12:12:12.012000000
11+
2022-03-01 12:12:12.012346
12+
2022-03-01 12:12:12.012
13+
2022-03-01 12:12:12.012300
14+
2022-03-01 12:12:12.012
15+
2022-03-01 12:12:12.012345670
16+
1950-03-01 12:12:12.012000000
17+
1951-03-01 12:12:12.012345
18+
1952-03-01 12:12:12.012
19+
1965-03-01 12:12:12.012300
20+
1966-03-01 12:12:12.012
21+
1967-03-01 12:12:12.012345670

tests/queries/0_stateless/02935_date_trunc_case_unsensitiveness.sql

Lines changed: 18 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -7,6 +7,21 @@ SELECT dateTrunc('Week', toDate('2022-03-01'));
77
SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55'));
88
SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2));
99
SELECT dateTrunc('week', toDate('2022-03-01'));
10-
SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError 36 }
11-
SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError 36 }
12-
SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError 36 }
10+
SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3));
11+
SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7));
12+
SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9));
13+
SELECT dateTrunc('mICROsECOND', toDateTime64('2022-03-01 12:12:12.0123', 4));
14+
SELECT dateTrunc('mIllISecoNd', toDateTime64('2022-03-01 12:12:12.0123456', 6));
15+
SELECT dateTrunc('NANoSecoND', toDateTime64('2022-03-01 12:12:12.012345678', 8));
16+
SELECT dateTrunc('Nanosecond', toDateTime64('1950-03-01 12:12:12.0123', 3));
17+
SELECT dateTrunc('MicroSecond', toDateTime64('1951-03-01 12:12:12.0123456', 7));
18+
SELECT dateTrunc('MILLISECOND', toDateTime64('1952-03-01 12:12:12.012324251', 9));
19+
SELECT dateTrunc('mICROsECOND', toDateTime64('1965-03-01 12:12:12.0123', 4));
20+
SELECT dateTrunc('mIllISecoNd', toDateTime64('1966-03-01 12:12:12.0123456', 6));
21+
SELECT dateTrunc('NANoSecoND', toDateTime64('1967-03-01 12:12:12.012345678', 8));
22+
SELECT dateTrunc('Nanosecond', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
23+
SELECT dateTrunc('MicroSecond', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
24+
SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
25+
SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
26+
SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
27+
SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

0 commit comments

Comments
 (0)