Merge branch 'master' into riscv-in-ci

This commit is contained in:
Alexey Milovidov 2022-08-22 01:37:53 +03:00 committed by GitHub
commit 8ca6025833
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 141 additions and 136 deletions

View File

@ -55,10 +55,9 @@ struct StringRef
bool empty() const { return size == 0; }
std::string toString() const { return std::string(data, size); }
explicit operator std::string() const { return toString(); }
std::string_view toView() const { return std::string_view(data, size); }
std::string_view toView() const { return std::string_view(data, size); }
constexpr explicit operator std::string_view() const { return std::string_view(data, size); }
};

View File

@ -16,7 +16,7 @@ option (ENABLE_SSE41 "Use SSE4.1 instructions on x86_64" 1)
option (ENABLE_SSE42 "Use SSE4.2 instructions on x86_64" 1)
option (ENABLE_PCLMULQDQ "Use pclmulqdq instructions on x86_64" 1)
option (ENABLE_POPCNT "Use popcnt instructions on x86_64" 1)
option (ENABLE_AVX "Use AVX instructions on x86_64" 1)
option (ENABLE_AVX "Use AVX instructions on x86_64" 0)
option (ENABLE_AVX2 "Use AVX2 instructions on x86_64" 0)
option (ENABLE_AVX512 "Use AVX512 instructions on x86_64" 0)
option (ENABLE_AVX512_VBMI "Use AVX512_VBMI instruction on x86_64 (depends on ENABLE_AVX512)" 0)

View File

@ -18,7 +18,9 @@ DateTime64(precision, [timezone])
Internally, stores data as a number of ticks since epoch start (1970-01-01 00:00:00 UTC) as Int64. The tick resolution is determined by the precision parameter. Additionally, the `DateTime64` type can store time zone that is the same for the entire column, that affects how the values of the `DateTime64` type values are displayed in text format and how the values specified as strings are parsed (2020-01-01 05:00:01.000). The time zone is not stored in the rows of the table (or in resultset), but is stored in the column metadata. See details in [DateTime](../../sql-reference/data-types/datetime.md).
Supported range of values: \[1900-01-01 00:00:00, 2299-12-31 23:59:59.99999999\] (Note: The precision of the maximum value is 8).
Supported range of values: \[1900-01-01 00:00:00, 2299-12-31 23:59:59.99999999\]
Note: The precision of the maximum value is 8. If the maximum precision of 9 digits (nanoseconds) is used, the maximum supported value is `2262-04-11 23:47:16` in UTC.
## Examples

View File

@ -884,12 +884,85 @@ Result:
└──────────────────────┘
```
## now64
Returns the current date and time with sub-second precision at the moment of query analysis. The function is a constant expression.
**Syntax**
``` sql
now64([scale], [timezone])
```
**Arguments**
- `scale` - Tick size (precision): 10<sup>-precision</sup> seconds. Valid range: [ 0 : 9 ]. Typically are used - 3 (default) (milliseconds), 6 (microseconds), 9 (nanoseconds).
- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md).
**Returned value**
- Current date and time with sub-second precision.
Type: [Datetime64](../../sql-reference/data-types/datetime64.md).
**Example**
``` sql
SELECT now64(), now64(9, 'Asia/Istanbul');
```
Result:
``` text
┌─────────────────now64()─┬─────now64(9, 'Asia/Istanbul')─┐
│ 2022-08-21 19:34:26.196 │ 2022-08-21 22:34:26.196542766 │
└─────────────────────────┴───────────────────────────────┘
```
## nowInBlock
Returns the current date and time at the moment of processing of each block of data. In contrast to the function `now`, it is not a constant expression, and the returned value will be different in different blocks for long-running queries.
Returns the current date and time at the moment of processing of each block of data. In contrast to the function [now](#now), it is not a constant expression, and the returned value will be different in different blocks for long-running queries.
It makes sense to use this function to generate the current time in long-running INSERT SELECT queries.
**Syntax**
``` sql
nowInBlock([timezone])
```
**Arguments**
- `timezone` — [Timezone name](../../operations/server-configuration-parameters/settings.md#server_configuration_parameters-timezone) for the returned value (optional). [String](../../sql-reference/data-types/string.md).
**Returned value**
- Current date and time at the moment of processing of each block of data.
Type: [Datetime](../../sql-reference/data-types/datetime.md).
**Example**
``` sql
SELECT
now(),
nowInBlock(),
sleep(1)
FROM numbers(3)
SETTINGS max_block_size = 1
FORMAT PrettyCompactMonoBlock
```
Result:
``` text
┌───────────────now()─┬────────nowInBlock()─┬─sleep(1)─┐
│ 2022-08-21 19:41:19 │ 2022-08-21 19:41:19 │ 0 │
│ 2022-08-21 19:41:19 │ 2022-08-21 19:41:20 │ 0 │
│ 2022-08-21 19:41:19 │ 2022-08-21 19:41:21 │ 0 │
└─────────────────────┴─────────────────────┴──────────┘
```
## today
Accepts zero arguments and returns the current date at one of the moments of query analysis.

View File

@ -444,6 +444,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
:::danger "Warning"
Не используйте в качестве источника ClickHouse, поскольку он медленно обрабатывает запросы со случайным чтением.
:::
### complex_key_cache {#complex-key-cache}

View File

@ -18,6 +18,9 @@ if(NOT CLICKHOUSE_ONE_SHARED)
target_link_libraries(clickhouse-local-lib PRIVATE clickhouse-server-lib)
endif()
# Always use internal readpassphrase
target_link_libraries(clickhouse-local-lib PRIVATE readpassphrase)
if (ENABLE_FUZZING)
add_compile_definitions(FUZZING_MODE=1)
set (WITH_COVERAGE ON)

View File

@ -191,29 +191,27 @@ struct ConvertImpl
vec_null_map_to = &col_null_map_to->getData();
}
if constexpr (std::is_same_v<ToDataType, DataTypeUInt8>)
bool result_is_bool = isBool(result_type);
for (size_t i = 0; i < input_rows_count; ++i)
{
if (isBool(result_type))
if constexpr (std::is_same_v<ToDataType, DataTypeUInt8>)
{
for (size_t i = 0; i < input_rows_count; ++i)
if (result_is_bool)
{
vec_to[i] = vec_from[i] != FromFieldType(0);
continue;
}
goto done;
}
}
if constexpr (std::is_same_v<FromDataType, DataTypeUUID> != std::is_same_v<ToDataType, DataTypeUUID>)
{
throw Exception("Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted", ErrorCodes::NOT_IMPLEMENTED);
}
else
{
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
if constexpr (std::is_same_v<FromDataType, DataTypeUUID> != std::is_same_v<ToDataType, DataTypeUUID>)
{
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
throw Exception("Conversion between numeric types and UUID is not supported. Probably the passed UUID is unquoted", ErrorCodes::NOT_IMPLEMENTED);
}
else
{
if constexpr (IsDataTypeDecimal<FromDataType> || IsDataTypeDecimal<ToDataType>)
{
for (size_t i = 0; i < input_rows_count; ++i)
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
{
ToFieldType result;
bool convert_result = false;
@ -233,10 +231,7 @@ struct ConvertImpl
(*vec_null_map_to)[i] = true;
}
}
}
else
{
for (size_t i = 0; i < input_rows_count; ++i)
else
{
if constexpr (IsDataTypeDecimal<FromDataType> && IsDataTypeDecimal<ToDataType>)
vec_to[i] = convertDecimals<FromDataType, ToDataType>(vec_from[i], col_from->getScale(), col_to->getScale());
@ -248,13 +243,10 @@ struct ConvertImpl
throw Exception("Unsupported data type in conversion function", ErrorCodes::CANNOT_CONVERT_TYPE);
}
}
}
else
{
/// If From Data is Nan or Inf and we convert to integer type, throw exception
if constexpr (std::is_floating_point_v<FromFieldType> && !std::is_floating_point_v<ToFieldType>)
else
{
for (size_t i = 0; i < input_rows_count; ++i)
/// If From Data is Nan or Inf and we convert to integer type, throw exception
if constexpr (std::is_floating_point_v<FromFieldType> && !std::is_floating_point_v<ToFieldType>)
{
if (!isFinite(vec_from[i]))
{
@ -262,46 +254,15 @@ struct ConvertImpl
{
vec_to[i] = 0;
(*vec_null_map_to)[i] = true;
continue;
}
else
throw Exception("Unexpected inf or nan to integer conversion", ErrorCodes::CANNOT_CONVERT_TYPE);
}
else
{
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>
|| std::is_same_v<Additions, AccurateConvertStrategyAdditions>)
{
bool convert_result = accurate::convertNumeric(vec_from[i], vec_to[i]);
if (!convert_result)
{
if (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
{
vec_to[i] = 0;
(*vec_null_map_to)[i] = true;
}
else
{
throw Exception(
"Value in column " + named_from.column->getName() + " cannot be safely converted into type "
+ result_type->getName(),
ErrorCodes::CANNOT_CONVERT_TYPE);
}
}
}
else
{
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
}
}
goto done;
}
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>
|| std::is_same_v<Additions, AccurateConvertStrategyAdditions>)
{
for (size_t i = 0; i < input_rows_count; ++i)
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>
|| std::is_same_v<Additions, AccurateConvertStrategyAdditions>)
{
bool convert_result = accurate::convertNumeric(vec_from[i], vec_to[i]);
@ -321,38 +282,14 @@ struct ConvertImpl
}
}
}
}
else
{
if constexpr (std::is_same_v<FromDataType, DataTypeUInt64> && std::is_same_v<ToDataType, DataTypeFloat32>)
{
/// Turns out that when ClickHouse is compiled with AVX1 or AVX2 instructions, Clang's autovectorizer produces
/// code for UInt64-to-Float23 conversion which is only ~50% as fast as scalar code. Interestingly, scalar code
/// is equally fast than code compiled for SSE4.2, so we might as well disable vectorization. This situation
/// may change with AVX512 which has a dediated instruction for that usecase (_mm512_cvtepi64_ps).
#if defined(__x86_64__)
# ifdef __clang__
# pragma clang loop vectorize(disable) interleave(disable)
# endif
#endif
for (size_t i = 0; i < input_rows_count; ++i)
{
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
}
else
{
for (size_t i = 0; i < input_rows_count; ++i)
{
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
vec_to[i] = static_cast<ToFieldType>(vec_from[i]);
}
}
}
}
done:
if constexpr (std::is_same_v<Additions, AccurateOrNullConvertStrategyAdditions>)
return ColumnNullable::create(std::move(col_to), std::move(col_null_map_to));
else

View File

@ -813,8 +813,8 @@ struct JSONExtractTree
auto from_col = dictionary_type->createColumn();
if (impl->insertResultToColumn(*from_col, element))
{
StringRef value = from_col->getDataAt(0);
assert_cast<ColumnLowCardinality &>(dest).insertData(value.data, value.size);
std::string_view value = from_col->getDataAt(0).toView();
assert_cast<ColumnLowCardinality &>(dest).insertData(value.data(), value.size());
return true;
}
return false;

View File

@ -11,19 +11,19 @@ struct ExtractTopLevelDomain
static void execute(Pos data, size_t size, Pos & res_data, size_t & res_size)
{
StringRef host = StringRef(getURLHost(data, size));
std::string_view host = getURLHost(data, size);
res_data = data;
res_size = 0;
if (host.size != 0)
if (!host.empty())
{
if (host.data[host.size - 1] == '.')
host.size -= 1;
if (host[host.size() - 1] == '.')
host.remove_suffix(1);
const auto * host_end = host.data + host.size;
const auto * host_end = host.data() + host.size();
Pos last_dot = find_last_symbols_or_null<'.'>(host.data, host_end);
Pos last_dot = find_last_symbols_or_null<'.'>(host.data(), host_end);
if (!last_dot)
return;

View File

@ -129,11 +129,11 @@ public:
root_offsets_data.resize(input_rows_count);
for (size_t i = 0; i < input_rows_count; ++i)
{
StringRef current_row = column_haystack->getDataAt(i);
std::string_view current_row = column_haystack->getDataAt(i).toView();
// Extract all non-intersecting matches from haystack except group #0.
const auto * pos = current_row.data;
const auto * end = pos + current_row.size;
const auto * pos = current_row.data();
const auto * end = pos + current_row.size();
while (pos < end
&& regexp->Match({pos, static_cast<size_t>(end - pos)},
0, end - pos, regexp->UNANCHORED, matched_groups.data(), matched_groups.size()))

View File

@ -94,14 +94,14 @@ struct RepeatImpl
template <typename T>
static void constStrVectorRepeat(
const StringRef & copy_str,
std::string_view copy_str,
ColumnString::Chars & res_data,
ColumnString::Offsets & res_offsets,
const PaddedPODArray<T> & col_num)
{
UInt64 data_size = 0;
res_offsets.resize(col_num.size());
UInt64 str_size = copy_str.size;
UInt64 str_size = copy_str.size();
UInt64 col_size = col_num.size();
for (UInt64 i = 0; i < col_size; ++i)
{
@ -116,7 +116,7 @@ struct RepeatImpl
T repeat_time = col_num[i];
checkRepeatTime(repeat_time);
process(
reinterpret_cast<UInt8 *>(const_cast<char *>(copy_str.data)),
reinterpret_cast<UInt8 *>(const_cast<char *>(copy_str.data())),
res_data.data() + res_offsets[i - 1],
str_size + 1,
repeat_time);
@ -227,7 +227,7 @@ public:
{
/// Note that const-const case is handled by useDefaultImplementationForConstants.
StringRef copy_str = col_const->getDataColumn().getDataAt(0);
std::string_view copy_str = col_const->getDataColumn().getDataAt(0).toView();
if (castType(arguments[1].type.get(), [&](const auto & type)
{

View File

@ -766,7 +766,7 @@ void DistributedSink::writeToShard(const Block & block, const std::vector<std::s
/// Write the header.
const std::string_view header = header_buf.stringView();
writeVarUInt(DBMS_DISTRIBUTED_SIGNATURE_HEADER, out);
writeStringBinary(StringRef(header), out);
writeStringBinary(header, out);
writePODBinary(CityHash_v1_0_2::CityHash128(header.data(), header.size()), out);
stream.write(block);

View File

@ -27,11 +27,11 @@ StorageXDBC::StorageXDBC(
const StorageID & table_id_,
const std::string & remote_database_name_,
const std::string & remote_table_name_,
const ColumnsDescription & columns_,
ColumnsDescription columns_,
ConstraintsDescription constraints_,
const String & comment,
ContextPtr context_,
const BridgeHelperPtr bridge_helper_)
/// Please add support for constraints as soon as StorageODBC or JDBC will support insertion.
: IStorageURLBase(
"",
context_,
@ -39,7 +39,7 @@ StorageXDBC::StorageXDBC(
IXDBCBridgeHelper::DEFAULT_FORMAT,
getFormatSettings(context_),
columns_,
ConstraintsDescription{},
constraints_,
comment,
"" /* CompressionMethod */)
, bridge_helper(bridge_helper_)
@ -179,6 +179,7 @@ namespace
checkAndGetLiteralArgument<String>(engine_args[1], "database_name"),
checkAndGetLiteralArgument<String>(engine_args[2], "table_name"),
args.columns,
args.constraints,
args.comment,
args.getContext(),
bridge_helper);

View File

@ -32,7 +32,8 @@ public:
const StorageID & table_id_,
const std::string & remote_database_name,
const std::string & remote_table_name,
const ColumnsDescription & columns_,
ColumnsDescription columns_,
ConstraintsDescription constraints_,
const String & comment,
ContextPtr context_,
BridgeHelperPtr bridge_helper_);

View File

@ -1,9 +1,6 @@
#include <type_traits>
#include <base/scope_guard.h>
#include <Core/Defines.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadWriteBufferFromHTTP.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Interpreters/evaluateConstantExpression.h>
@ -16,10 +13,9 @@
#include <TableFunctions/TableFunctionFactory.h>
#include <Poco/Net/HTTPRequest.h>
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <Poco/NumberFormatter.h>
#include "registerTableFunctions.h"
namespace DB
{
namespace ErrorCodes
@ -33,13 +29,12 @@ void ITableFunctionXDBC::parseArguments(const ASTPtr & ast_function, ContextPtr
const auto & args_func = ast_function->as<ASTFunction &>();
if (!args_func.arguments)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR);
throw Exception(ErrorCodes::LOGICAL_ERROR, "Table function '{}' must have arguments.", getName());
ASTs & args = args_func.arguments->children;
if (args.size() != 2 && args.size() != 3)
throw Exception("Table function '" + getName() + "' requires 2 or 3 arguments: " + getName() + "('DSN', table) or " + getName()
+ "('DSN', schema, table)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Table function '{0}' requires 2 or 3 arguments: {0}('DSN', table) or {0}('DSN', schema, table)", getName());
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
@ -61,7 +56,6 @@ void ITableFunctionXDBC::startBridgeIfNot(ContextPtr context) const
{
if (!helper)
{
/// Have to const_cast, because bridges store their commands inside context
helper = createBridgeHelper(context, context->getSettingsRef().http_receive_timeout.value, connection_string);
helper->startBridgeSync();
}
@ -71,16 +65,15 @@ ColumnsDescription ITableFunctionXDBC::getActualTableStructure(ContextPtr contex
{
startBridgeIfNot(context);
/* Infer external table structure */
/// Infer external table structure.
Poco::URI columns_info_uri = helper->getColumnsInfoURI();
columns_info_uri.addQueryParameter("connection_string", connection_string);
if (!schema_name.empty())
columns_info_uri.addQueryParameter("schema", schema_name);
columns_info_uri.addQueryParameter("table", remote_table_name);
const auto use_nulls = context->getSettingsRef().external_table_functions_use_nulls;
columns_info_uri.addQueryParameter("external_table_functions_use_nulls",
Poco::NumberFormatter::format(use_nulls));
bool use_nulls = context->getSettingsRef().external_table_functions_use_nulls;
columns_info_uri.addQueryParameter("external_table_functions_use_nulls", toString(use_nulls));
Poco::Net::HTTPBasicCredentials credentials{};
ReadWriteBufferFromHTTP buf(columns_info_uri, Poco::Net::HTTPRequest::HTTP_POST, {}, ConnectionTimeouts::getHTTPTimeouts(context), credentials);
@ -97,7 +90,7 @@ StoragePtr ITableFunctionXDBC::executeImpl(const ASTPtr & /*ast_function*/, Cont
startBridgeIfNot(context);
auto columns = getActualTableStructure(context);
auto result = std::make_shared<StorageXDBC>(
StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, String{}, context, helper);
StorageID(getDatabaseName(), table_name), schema_name, remote_table_name, columns, ConstraintsDescription{}, String{}, context, helper);
result->startup();
return result;
}

View File

@ -0,0 +1,7 @@
Instruction check fail. The CPU does not support SSSE3 instruction set.
Instruction check fail. The CPU does not support SSE4.1 instruction set.
Instruction check fail. The CPU does not support SSE4.2 instruction set.
Instruction check fail. The CPU does not support POPCNT instruction set.
<jemalloc>: MADV_DONTNEED does not work (memset will be used instead)
<jemalloc>: (This is the expected behaviour if you are running under QEMU)
1

View File

@ -2,18 +2,6 @@
# Tags: no-tsan, no-asan, no-ubsan, no-msan, no-debug, no-fasttest, no-cpu-aarch64
# Tag no-fasttest: avoid dependency on qemu -- invonvenient when running locally
# More than a decade after AVX was released, AVX is still not supported by QEMU, even if "-cpu help" pretends to. As a result, we cannot use
# QEMU to verify that a ClickHouse binary compiled for a SIMD level up to AVX runs on a system with a SIMD level up to AVX. The alternative
# is to disassemble the binary and grep for unwanted instructions (e.g. AVX512) which is just too fragile ...
#
# https://gitlab.com/qemu-project/qemu/-/issues/164
# https://www.mail-archive.com/qemu-devel@nongnu.org/msg713932.html
# https://lore.kernel.org/all/CAObpvQmejWBh+RNz2vhk16-kcY_QveM_pSmM5ZeWqWv1d8AJzQ@mail.gmail.com/T/
exit 0
# keeping the original test because it is instructive and maybe QEMU will be fixed at some point ...
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh