Merge remote-tracking branch 'rschu1ze/master' into bump-azure-1.8

This commit is contained in:
Robert Schulze 2024-04-17 08:14:20 +00:00
commit 0fbfe24f73
No known key found for this signature in database
GPG Key ID: 26703B55FB13728A
12 changed files with 109 additions and 28 deletions

View File

@ -100,8 +100,9 @@ if(ARCH_AMD64)
set(OPENSSL_SYSTEM "macosx")
else()
macro(perl_generate_asm FILE_IN FILE_OUT)
# Manually set $CC because the called Perl scripts require it.
add_custom_command(OUTPUT ${FILE_OUT}
COMMAND /usr/bin/env perl ${FILE_IN} ${FILE_OUT})
COMMAND ${CMAKE_COMMAND} -E env "CC=${CMAKE_CXX_COMPILER}" /usr/bin/env perl ${FILE_IN} ${FILE_OUT})
endmacro()
perl_generate_asm(${OPENSSL_SOURCE_DIR}/crypto/aes/asm/aes-x86_64.pl ${OPENSSL_BINARY_DIR}/crypto/aes/aes-x86_64.s)

View File

@ -1836,6 +1836,9 @@ Alias: `dateTrunc`.
- `unit` — The type of interval to truncate the result. [String Literal](../syntax.md#syntax-string-literal).
Possible values:
- `nanosecond` - Compatible only with DateTime64
- `microsecond` - Compatible only with DateTime64
- `milisecond` - Compatible only with DateTime64
- `second`
- `minute`
- `hour`

View File

@ -40,12 +40,16 @@ struct LargestTriangleThreeBucketsData : public StatisticalSample<Float64, Float
{
void add(const Float64 xval, const Float64 yval, Arena * arena)
{
/// We need to ensure either both or neither coordinates are saved (StatisticalSample ignores NaNs)
if (isNaN(xval) || isNaN(yval))
return;
this->addX(xval, arena);
this->addY(yval, arena);
}
void sort(Arena * arena)
{
chassert(this->x.size() == this->y.size());
// sort the this->x and this->y in ascending order of this->x using index
std::vector<size_t> index(this->x.size());

View File

@ -67,7 +67,7 @@ namespace
return date_time_type;
}
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr & result_type, size_t) const override
ColumnPtr executeImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &, size_t) const override
{
if (arguments.size() != 2)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Function {}'s arguments number must be 2.", name);
@ -77,37 +77,47 @@ namespace
if (!time_zone_const_col)
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Illegal column {} of 2nd argument of function {}. Excepted const(String).", arg2.column->getName(), name);
String time_zone_val = time_zone_const_col->getDataAt(0).toString();
auto column = result_type->createColumn();
const DateLUTImpl & utc_time_zone = DateLUT::instance("UTC");
if (WhichDataType(arg1.type).isDateTime())
{
const auto * date_time_col = checkAndGetColumn<ColumnDateTime>(arg1.column.get());
for (size_t i = 0; i < date_time_col->size(); ++i)
size_t col_size = date_time_col->size();
using ColVecTo = DataTypeDateTime::ColumnType;
typename ColVecTo::MutablePtr result_column = ColVecTo::create(col_size);
typename ColVecTo::Container & result_data = result_column->getData();
for (size_t i = 0; i < col_size; ++i)
{
UInt32 date_time_val = date_time_col->getElement(i);
LocalDateTime date_time(date_time_val, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
column->insert(time_val);
LocalDateTime date_time(date_time_val, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));
time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val));
result_data[i] = static_cast<UInt32>(time_val);
}
return result_column;
}
else if (WhichDataType(arg1.type).isDateTime64())
{
const auto * date_time_col = checkAndGetColumn<ColumnDateTime64>(arg1.column.get());
size_t col_size = date_time_col->size();
const DataTypeDateTime64 * date_time_type = static_cast<const DataTypeDateTime64 *>(arg1.type.get());
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(date_time_type->getScale());
for (size_t i = 0; i < date_time_col->size(); ++i)
UInt32 col_scale = date_time_type->getScale();
Int64 scale_multiplier = DecimalUtils::scaleMultiplier<Int64>(col_scale);
using ColDecimalTo = DataTypeDateTime64::ColumnType;
typename ColDecimalTo::MutablePtr result_column = ColDecimalTo::create(col_size, col_scale);
typename ColDecimalTo::Container & result_data = result_column->getData();
for (size_t i = 0; i < col_size; ++i)
{
DateTime64 date_time_val = date_time_col->getElement(i);
Int64 seconds = date_time_val.value / scale_multiplier;
Int64 micros = date_time_val.value % scale_multiplier;
LocalDateTime date_time(seconds, Name::to ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
time_t time_val = date_time.to_time_t(Name::from ? DateLUT::instance("UTC") : DateLUT::instance(time_zone_val));
LocalDateTime date_time(seconds, Name::to ? utc_time_zone : DateLUT::instance(time_zone_val));
time_t time_val = date_time.to_time_t(Name::from ? utc_time_zone : DateLUT::instance(time_zone_val));
DateTime64 date_time_64(time_val * scale_multiplier + micros);
column->insert(date_time_64);
result_data[i] = date_time_64;
}
return result_column;
}
else
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT, "Function {}'s 1st argument can only be datetime/datatime64. ", name);
return column;
}
};

View File

@ -40,7 +40,14 @@ public:
{
/// The first argument is a constant string with the name of datepart.
auto result_type_is_date = false;
enum ResultType
{
Date,
DateTime,
DateTime64,
};
ResultType result_type;
String datepart_param;
auto check_first_argument = [&] {
const ColumnConst * datepart_column = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
@ -56,13 +63,14 @@ public:
if (!IntervalKind::tryParseString(datepart_param, datepart_kind))
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't look like datepart name in {}", datepart_param, getName());
if (datepart_kind == IntervalKind::Kind::Nanosecond || datepart_kind == IntervalKind::Kind::Microsecond
|| datepart_kind == IntervalKind::Kind::Millisecond)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} doesn't support {}", getName(), datepart_param);
result_type_is_date = (datepart_kind == IntervalKind::Kind::Year)
|| (datepart_kind == IntervalKind::Kind::Quarter) || (datepart_kind == IntervalKind::Kind::Month)
|| (datepart_kind == IntervalKind::Kind::Week);
if ((datepart_kind == IntervalKind::Kind::Year) || (datepart_kind == IntervalKind::Kind::Quarter)
|| (datepart_kind == IntervalKind::Kind::Month) || (datepart_kind == IntervalKind::Kind::Week))
result_type = ResultType::Date;
else if ((datepart_kind == IntervalKind::Kind::Day) || (datepart_kind == IntervalKind::Kind::Hour)
|| (datepart_kind == IntervalKind::Kind::Minute) || (datepart_kind == IntervalKind::Kind::Second))
result_type = ResultType::DateTime;
else
result_type = ResultType::DateTime64;
};
bool second_argument_is_date = false;
@ -84,7 +92,7 @@ public:
"This argument is optional and must be a constant string with timezone name",
arguments[2].type->getName(), getName());
if (second_argument_is_date && result_type_is_date)
if (second_argument_is_date && result_type == ResultType::Date)
throw Exception(ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT,
"The timezone argument of function {} with datepart '{}' "
"is allowed only when the 2nd argument has the type DateTime",
@ -109,10 +117,21 @@ public:
getName(), arguments.size());
}
if (result_type_is_date)
if (result_type == ResultType::Date)
return std::make_shared<DataTypeDate>();
else
else if (result_type == ResultType::DateTime)
return std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
else
{
size_t scale;
if (datepart_kind == IntervalKind::Kind::Millisecond)
scale = 3;
else if (datepart_kind == IntervalKind::Kind::Microsecond)
scale = 6;
else if (datepart_kind == IntervalKind::Kind::Nanosecond)
scale = 9;
return std::make_shared<DataTypeDateTime64>(scale, extractTimeZoneNameFromFunctionArguments(arguments, 2, 1, false));
}
}
bool useDefaultImplementationForConstants() const override { return true; }

View File

@ -7,6 +7,7 @@
#include <IO/ReadBufferFromString.h>
#include <Common/Exception.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/threadPoolCallbackRunner.h>
#include <Core/ServerUUID.h>
#include <Common/logger_useful.h>
#include <Common/noexcept_scope.h>
@ -451,6 +452,7 @@ CSN TransactionLog::commitTransaction(const MergeTreeTransactionPtr & txn, bool
CSN TransactionLog::finalizeCommittedTransaction(MergeTreeTransaction * txn, CSN allocated_csn, scope_guard & state_guard) noexcept
{
LockMemoryExceptionInThread memory_tracker_lock(VariableContext::Global);
auto blocker = CannotAllocateThreadFaultInjector::blockFaultInjections();
chassert(!allocated_csn == txn->isReadOnly());
if (allocated_csn)
{

View File

@ -67,5 +67,4 @@ from
select throwIf(uniq((test, query)) != 1) from table
) check_single_query -- this subselect checks that there is only one query in the input table;
-- written this way so that it is not optimized away (#10523)
SETTINGS allow_experimental_analyzer = 0
;

View File

@ -0,0 +1,14 @@
<test>
<create_query>CREATE TABLE test1(d DateTime) ENGINE Memory</create_query>
<create_query>CREATE TABLE test2(d DateTime64) ENGINE Memory</create_query>
<fill_query>INSERT INTO test1 SELECT toDateTime('2023-03-16 11:22:33') + number from numbers(10000000)</fill_query>
<fill_query>INSERT INTO test2 SELECT toDateTime64('2023-03-16 11:22:33', 3) + number from numbers(10000000)</fill_query>
<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>
<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>
<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>
<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>
<drop_query>DROP TABLE IF EXISTS test1</drop_query>
<drop_query>DROP TABLE IF EXISTS test2</drop_query>
</test>

View File

@ -7,3 +7,15 @@
2022-03-01 00:00:00
2022-03-01
2022-02-28
2022-03-01 12:12:12.012000000
2022-03-01 12:12:12.012346
2022-03-01 12:12:12.012
2022-03-01 12:12:12.012300
2022-03-01 12:12:12.012
2022-03-01 12:12:12.012345670
1950-03-01 12:12:12.012000000
1951-03-01 12:12:12.012345
1952-03-01 12:12:12.012
1965-03-01 12:12:12.012300
1966-03-01 12:12:12.012
1967-03-01 12:12:12.012345670

View File

@ -7,6 +7,21 @@ SELECT dateTrunc('Week', toDate('2022-03-01'));
SELECT dateTrunc('day', toDateTime('2022-03-01 12:55:55'));
SELECT dateTrunc('month', toDateTime64('2022-03-01 12:55:55', 2));
SELECT dateTrunc('week', toDate('2022-03-01'));
SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError 36 }
SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError 36 }
SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError 36 }
SELECT dateTrunc('Nanosecond', toDateTime64('2022-03-01 12:12:12.0123', 3));
SELECT dateTrunc('MicroSecond', toDateTime64('2022-03-01 12:12:12.0123456', 7));
SELECT dateTrunc('MILLISECOND', toDateTime64('2022-03-01 12:12:12.012324251', 9));
SELECT dateTrunc('mICROsECOND', toDateTime64('2022-03-01 12:12:12.0123', 4));
SELECT dateTrunc('mIllISecoNd', toDateTime64('2022-03-01 12:12:12.0123456', 6));
SELECT dateTrunc('NANoSecoND', toDateTime64('2022-03-01 12:12:12.012345678', 8));
SELECT dateTrunc('Nanosecond', toDateTime64('1950-03-01 12:12:12.0123', 3));
SELECT dateTrunc('MicroSecond', toDateTime64('1951-03-01 12:12:12.0123456', 7));
SELECT dateTrunc('MILLISECOND', toDateTime64('1952-03-01 12:12:12.012324251', 9));
SELECT dateTrunc('mICROsECOND', toDateTime64('1965-03-01 12:12:12.0123', 4));
SELECT dateTrunc('mIllISecoNd', toDateTime64('1966-03-01 12:12:12.0123456', 6));
SELECT dateTrunc('NANoSecoND', toDateTime64('1967-03-01 12:12:12.012345678', 8));
SELECT dateTrunc('Nanosecond', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MicroSecond', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MILLISECOND', toDateTime('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('Nanosecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MicroSecond', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }
SELECT dateTrunc('MILLISECOND', toDate('2022-03-01')); -- { serverError ILLEGAL_TYPE_OF_ARGUMENT }

View File

@ -0,0 +1 @@
SELECT largestTriangleThreeBuckets(1)(1, nan);