mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Merge branch 'master' into riscv-in-ci
This commit is contained in:
commit
8ca6025833
@ -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); }
|
||||
};
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
||||
|
@ -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.
|
||||
|
@ -444,6 +444,7 @@ LAYOUT(CACHE(SIZE_IN_CELLS 1000000000))
|
||||
|
||||
:::danger "Warning"
|
||||
Не используйте в качестве источника ClickHouse, поскольку он медленно обрабатывает запросы со случайным чтением.
|
||||
:::
|
||||
|
||||
### complex_key_cache {#complex-key-cache}
|
||||
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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()))
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
|
@ -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_);
|
||||
|
@ -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;
|
||||
}
|
||||
|
@ -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
|
@ -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
|
||||
|
Loading…
Reference in New Issue
Block a user