mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge remote-tracking branch 'rschu1ze/master' into bump-azure-1.8
This commit is contained in:
commit
0fbfe24f73
@ -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)
|
||||
|
@ -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`
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
};
|
||||
|
@ -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; }
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
;
|
||||
|
14
tests/performance/utc_timestamp_transform.xml
Normal file
14
tests/performance/utc_timestamp_transform.xml
Normal 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>
|
@ -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
|
||||
|
@ -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 }
|
||||
|
@ -0,0 +1 @@
|
||||
[]
|
@ -0,0 +1 @@
|
||||
SELECT largestTriangleThreeBuckets(1)(1, nan);
|
Loading…
Reference in New Issue
Block a user