mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-29 19:12:03 +00:00
Merge remote-tracking branch 'origin/master' into use-iobject-storage-for-table-engines-1
This commit is contained in:
commit
e23b929e18
@ -1,6 +1,7 @@
|
||||
#pragma once
|
||||
|
||||
#include <base/types.h>
|
||||
#include <base/extended_types.h>
|
||||
|
||||
namespace wide
|
||||
{
|
||||
@ -44,3 +45,8 @@ concept is_over_big_int =
|
||||
|| std::is_same_v<T, Decimal128>
|
||||
|| std::is_same_v<T, Decimal256>;
|
||||
}
|
||||
|
||||
template <> struct is_signed<DB::Decimal32> { static constexpr bool value = true; };
|
||||
template <> struct is_signed<DB::Decimal64> { static constexpr bool value = true; };
|
||||
template <> struct is_signed<DB::Decimal128> { static constexpr bool value = true; };
|
||||
template <> struct is_signed<DB::Decimal256> { static constexpr bool value = true; };
|
||||
|
2
contrib/curl
vendored
2
contrib/curl
vendored
@ -1 +1 @@
|
||||
Subproject commit 7161cb17c01dcff1dc5bf89a18437d9d729f1ecd
|
||||
Subproject commit 5ce164e0e9290c96eb7d502173426c0a135ec008
|
2
contrib/lz4
vendored
2
contrib/lz4
vendored
@ -1 +1 @@
|
||||
Subproject commit 92ebf1870b9acbefc0e7970409a181954a10ff40
|
||||
Subproject commit ce45a9dbdb059511a3e9576b19db3e7f1a4f172e
|
@ -190,7 +190,7 @@ function setup_logs_replication
|
||||
echo -e "Creating remote destination table ${table}_${hash} with statement:\n${statement}" >&2
|
||||
|
||||
echo "$statement" | clickhouse-client --database_replicated_initial_query_timeout_sec=10 \
|
||||
--distributed_ddl_task_timeout=30 \
|
||||
--distributed_ddl_task_timeout=30 --distributed_ddl_output_mode=throw_only_active \
|
||||
"${CONNECTION_ARGS[@]}" || continue
|
||||
|
||||
echo "Creating table system.${table}_sender" >&2
|
||||
|
@ -204,6 +204,20 @@ Result:
|
||||
└─────┴───────────────────────┘
|
||||
```
|
||||
|
||||
## Inserts into ClickHouse Cloud
|
||||
|
||||
By default, services on ClickHouse Cloud provide multiple replicas for high availability. When you connect to a service, a connection is established to one of these replicas.
|
||||
|
||||
After an `INSERT` succeeds, data is written to the underlying storage. However, it may take some time for replicas to receive these updates. Therefore, if you use a different connection that executes a `SELECT` query on one of these other replicas, the updated data may not yet be reflected.
|
||||
|
||||
It is possible to use the `select_sequential_consistency` to force the replica to receive the latest updates. Here is an example of a SELECT query using this setting:
|
||||
|
||||
```sql
|
||||
SELECT .... SETTINGS select_sequential_consistency = 1;
|
||||
```
|
||||
|
||||
Note that using `select_sequential_consistency` will increase the load on ClickHouse Keeper (used by ClickHouse Cloud internally) and may result in slower performance depending on the load on the service. We recommend against enabling this setting unless necessary. The recommended approach is to execute read/writes in the same session or to use a client driver that uses the native protocol (and thus supports sticky connections).
|
||||
|
||||
## Performance Considerations
|
||||
|
||||
`INSERT` sorts the input data by primary key and splits them into partitions by a partition key. If you insert data into several partitions at once, it can significantly reduce the performance of the `INSERT` query. To avoid this:
|
||||
|
@ -845,83 +845,7 @@ bool Client::processWithFuzzing(const String & full_query)
|
||||
have_error = true;
|
||||
}
|
||||
|
||||
// Check that after the query is formatted, we can parse it back,
|
||||
// format again and get the same result. Unfortunately, we can't
|
||||
// compare the ASTs, which would be more sensitive to errors. This
|
||||
// double formatting check doesn't catch all errors, e.g. we can
|
||||
// format query incorrectly, but to a valid SQL that we can then
|
||||
// parse and format into the same SQL.
|
||||
// There are some complicated cases where we can generate the SQL
|
||||
// which we can't parse:
|
||||
// * first argument of lambda() replaced by fuzzer with
|
||||
// something else, leading to constructs such as
|
||||
// arrayMap((min(x) + 3) -> x + 1, ....)
|
||||
// * internals of Enum replaced, leading to:
|
||||
// Enum(equals(someFunction(y), 3)).
|
||||
// And there are even the cases when we can parse the query, but
|
||||
// it's logically incorrect and its formatting is a mess, such as
|
||||
// when `lambda()` function gets substituted into a wrong place.
|
||||
// To avoid dealing with these cases, run the check only for the
|
||||
// queries we were able to successfully execute.
|
||||
// Another caveat is that sometimes WITH queries are not executed,
|
||||
// if they are not referenced by the main SELECT, so they can still
|
||||
// have the aforementioned problems. Disable this check for such
|
||||
// queries, for lack of a better solution.
|
||||
// There is also a problem that fuzzer substitutes positive Int64
|
||||
// literals or Decimal literals, which are then parsed back as
|
||||
// UInt64, and suddenly duplicate alias substitution starts or stops
|
||||
// working (ASTWithAlias::formatImpl) or something like that.
|
||||
// So we compare not even the first and second formatting of the
|
||||
// query, but second and third.
|
||||
// If you have to add any more workarounds to this check, just remove
|
||||
// it altogether, it's not so useful.
|
||||
if (ast_to_process && !have_error && !queryHasWithClause(*ast_to_process))
|
||||
{
|
||||
ASTPtr ast_2;
|
||||
try
|
||||
{
|
||||
const auto * tmp_pos = query_to_execute.c_str();
|
||||
ast_2 = parseQuery(tmp_pos, tmp_pos + query_to_execute.size(), false /* allow_multi_statements */);
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
if (e.code() != ErrorCodes::SYNTAX_ERROR &&
|
||||
e.code() != ErrorCodes::TOO_DEEP_RECURSION)
|
||||
throw;
|
||||
}
|
||||
|
||||
if (ast_2)
|
||||
{
|
||||
const auto text_2 = ast_2->formatForErrorMessage();
|
||||
const auto * tmp_pos = text_2.c_str();
|
||||
const auto ast_3 = parseQuery(tmp_pos, tmp_pos + text_2.size(),
|
||||
false /* allow_multi_statements */);
|
||||
const auto text_3 = ast_3 ? ast_3->formatForErrorMessage() : "";
|
||||
|
||||
if (text_3 != text_2)
|
||||
{
|
||||
fmt::print(stderr, "Found error: The query formatting is broken.\n");
|
||||
|
||||
printChangedSettings();
|
||||
|
||||
fmt::print(stderr,
|
||||
"Got the following (different) text after formatting the fuzzed query and parsing it back:\n'{}'\n, expected:\n'{}'\n",
|
||||
text_3, text_2);
|
||||
fmt::print(stderr, "In more detail:\n");
|
||||
fmt::print(stderr, "AST-1 (generated by fuzzer):\n'{}'\n", ast_to_process->dumpTree());
|
||||
fmt::print(stderr, "Text-1 (AST-1 formatted):\n'{}'\n", query_to_execute);
|
||||
fmt::print(stderr, "AST-2 (Text-1 parsed):\n'{}'\n", ast_2->dumpTree());
|
||||
fmt::print(stderr, "Text-2 (AST-2 formatted):\n'{}'\n", text_2);
|
||||
fmt::print(stderr, "AST-3 (Text-2 parsed):\n'{}'\n", ast_3 ? ast_3->dumpTree() : "");
|
||||
fmt::print(stderr, "Text-3 (AST-3 formatted):\n'{}'\n", text_3);
|
||||
fmt::print(stderr, "Text-3 must be equal to Text-2, but it is not.\n");
|
||||
|
||||
_exit(1);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// The server is still alive so we're going to continue fuzzing.
|
||||
// The server is still alive, so we're going to continue fuzzing.
|
||||
// Determine what we're going to use as the starting AST.
|
||||
if (have_error)
|
||||
{
|
||||
|
@ -44,6 +44,7 @@
|
||||
#include <Common/assertProcessUserMatchesDataOwner.h>
|
||||
#include <Common/makeSocketAddress.h>
|
||||
#include <Common/FailPoint.h>
|
||||
#include <Common/CPUID.h>
|
||||
#include <Server/waitServersToFinish.h>
|
||||
#include <Interpreters/Cache/FileCacheFactory.h>
|
||||
#include <Core/ServerUUID.h>
|
||||
@ -712,6 +713,22 @@ try
|
||||
getNumberOfPhysicalCPUCores(), // on ARM processors it can show only enabled at current moment cores
|
||||
std::thread::hardware_concurrency());
|
||||
|
||||
#if defined(__x86_64__)
|
||||
String cpu_info;
|
||||
#define COLLECT_FLAG(X) \
|
||||
if (CPU::have##X()) \
|
||||
{ \
|
||||
if (!cpu_info.empty()) \
|
||||
cpu_info += ", "; \
|
||||
cpu_info += #X; \
|
||||
}
|
||||
|
||||
CPU_ID_ENUMERATE(COLLECT_FLAG)
|
||||
#undef COLLECT_FLAG
|
||||
|
||||
LOG_INFO(log, "Available CPU instruction sets: {}", cpu_info);
|
||||
#endif
|
||||
|
||||
sanityChecks(*this);
|
||||
|
||||
// Initialize global thread pool. Do it before we fetch configs from zookeeper
|
||||
|
@ -219,7 +219,7 @@ public:
|
||||
: IAggregateFunctionDataHelper<AggregateFunctionCountData, AggregateFunctionCountNotNullUnary>({argument}, params, createResultType())
|
||||
{
|
||||
if (!argument->isNullable())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: not Nullable data type passed to AggregateFunctionCountNotNullUnary");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Not Nullable data type passed to AggregateFunctionCountNotNullUnary");
|
||||
}
|
||||
|
||||
String getName() const override { return "count"; }
|
||||
|
@ -100,7 +100,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
{
|
||||
AggregateFunctionCombinatorPtr combinator = AggregateFunctionCombinatorFactory::instance().tryFindSuffix("Null");
|
||||
if (!combinator)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: cannot find aggregate function combinator "
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot find aggregate function combinator "
|
||||
"to apply a function to Nullable arguments.");
|
||||
|
||||
DataTypes nested_types = combinator->transformArguments(types_without_low_cardinality);
|
||||
@ -123,7 +123,7 @@ AggregateFunctionPtr AggregateFunctionFactory::get(
|
||||
auto with_original_arguments = getImpl(name, action, types_without_low_cardinality, parameters, out_properties, false);
|
||||
|
||||
if (!with_original_arguments)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: AggregateFunctionFactory returned nullptr");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "AggregateFunctionFactory returned nullptr");
|
||||
return with_original_arguments;
|
||||
}
|
||||
|
||||
|
@ -249,7 +249,7 @@ public:
|
||||
: Base(std::move(nested_function_), arguments, params), number_of_arguments(arguments.size())
|
||||
{
|
||||
if (number_of_arguments == 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionIfNullVariadic");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionIfNullVariadic");
|
||||
|
||||
if (number_of_arguments > MAX_ARGS)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
|
@ -429,7 +429,7 @@ public:
|
||||
, number_of_arguments(arguments.size())
|
||||
{
|
||||
if (number_of_arguments == 1)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: single argument is passed to AggregateFunctionNullVariadic");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Single argument is passed to AggregateFunctionNullVariadic");
|
||||
|
||||
if (number_of_arguments > MAX_ARGS)
|
||||
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
||||
|
@ -79,8 +79,6 @@
|
||||
#include <Analyzer/QueryTreeBuilder.h>
|
||||
#include <Analyzer/IQueryTreeNode.h>
|
||||
#include <Analyzer/Identifier.h>
|
||||
#include <Poco/Logger.h>
|
||||
#include <Common/logger_useful.h>
|
||||
|
||||
namespace ProfileEvents
|
||||
{
|
||||
@ -6737,6 +6735,28 @@ void QueryAnalyzer::resolveTableFunction(QueryTreeNodePtr & table_function_node,
|
||||
TableFunctionPtr table_function_ptr = TableFunctionFactory::instance().tryGet(table_function_name, scope_context);
|
||||
if (!table_function_ptr)
|
||||
{
|
||||
String database_name = scope_context->getCurrentDatabase();
|
||||
String table_name;
|
||||
|
||||
auto function_ast = table_function_node->toAST();
|
||||
Identifier table_identifier{table_function_name};
|
||||
if (table_identifier.getPartsSize() == 1)
|
||||
{
|
||||
table_name = table_identifier[0];
|
||||
}
|
||||
else if (table_identifier.getPartsSize() == 2)
|
||||
{
|
||||
database_name = table_identifier[0];
|
||||
table_name = table_identifier[1];
|
||||
}
|
||||
|
||||
auto parametrized_view_storage = scope_context->getQueryContext()->buildParametrizedViewStorage(function_ast, database_name, table_name);
|
||||
if (parametrized_view_storage)
|
||||
{
|
||||
table_function_node = std::make_shared<TableNode>(parametrized_view_storage, scope_context);
|
||||
return;
|
||||
}
|
||||
|
||||
auto hints = TableFunctionFactory::instance().getHints(table_function_name);
|
||||
if (!hints.empty())
|
||||
throw Exception(ErrorCodes::UNKNOWN_FUNCTION,
|
||||
|
@ -421,11 +421,8 @@ ASTPtr QueryNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
|
||||
if (is_subquery)
|
||||
{
|
||||
auto subquery = std::make_shared<ASTSubquery>();
|
||||
|
||||
auto subquery = std::make_shared<ASTSubquery>(std::move(result_select_query));
|
||||
subquery->cte_name = cte_name;
|
||||
subquery->children.push_back(std::move(result_select_query));
|
||||
|
||||
return subquery;
|
||||
}
|
||||
|
||||
|
@ -185,11 +185,8 @@ ASTPtr UnionNode::toASTImpl(const ConvertToASTOptions & options) const
|
||||
|
||||
if (is_subquery)
|
||||
{
|
||||
auto subquery = std::make_shared<ASTSubquery>();
|
||||
|
||||
auto subquery = std::make_shared<ASTSubquery>(std::move(select_with_union_query));
|
||||
subquery->cte_name = cte_name;
|
||||
subquery->children.push_back(std::move(select_with_union_query));
|
||||
|
||||
return subquery;
|
||||
}
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
#include <IO/HTTPHeaderEntries.h>
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#if USE_AWS_S3
|
||||
#include <Common/quoteString.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <IO/SharedThreadPools.h>
|
||||
#include <IO/ReadBufferFromS3.h>
|
||||
|
@ -3,7 +3,6 @@
|
||||
#include <Common/AsyncTaskExecutor.h>
|
||||
#include <Common/Epoll.h>
|
||||
#include <Common/Fiber.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/TimerDescriptor.h>
|
||||
#include <Common/PoolWithFailoverBase.h>
|
||||
#include <Client/ConnectionPool.h>
|
||||
|
@ -125,7 +125,7 @@ ConnectionParameters::ConnectionParameters(const Poco::Util::AbstractConfigurati
|
||||
Poco::Timespan(config.getInt("send_timeout", DBMS_DEFAULT_SEND_TIMEOUT_SEC), 0))
|
||||
.withReceiveTimeout(
|
||||
Poco::Timespan(config.getInt("receive_timeout", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC), 0))
|
||||
.withTcpKeepAliveTimeout(
|
||||
.withTCPKeepAliveTimeout(
|
||||
Poco::Timespan(config.getInt("tcp_keep_alive_timeout", DEFAULT_TCP_KEEP_ALIVE_TIMEOUT), 0))
|
||||
.withHandshakeTimeout(
|
||||
Poco::Timespan(config.getInt("handshake_timeout_ms", DBMS_DEFAULT_RECEIVE_TIMEOUT_SEC * 1000) * 1000))
|
||||
|
@ -320,7 +320,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
|
||||
ReplicaState & state = getReplicaForReading();
|
||||
current_connection = state.connection;
|
||||
if (current_connection == nullptr)
|
||||
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "Logical error: no available replica");
|
||||
throw Exception(ErrorCodes::NO_AVAILABLE_REPLICA, "No available replica");
|
||||
|
||||
Packet packet;
|
||||
try
|
||||
|
@ -5,7 +5,6 @@
|
||||
#include <variant>
|
||||
|
||||
#include <Client/IConnections.h>
|
||||
#include <Common/FiberStack.h>
|
||||
#include <Common/Fiber.h>
|
||||
#include <Common/Epoll.h>
|
||||
#include <Common/TimerDescriptor.h>
|
||||
|
@ -1168,23 +1168,13 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
|
||||
|
||||
fuzz(select->children);
|
||||
}
|
||||
/*
|
||||
* The time to fuzz the settings has not yet come.
|
||||
* Apparently we don't have any infrastructure to validate the values of
|
||||
* the settings, and the first query with max_block_size = -1 breaks
|
||||
* because of overflows here and there.
|
||||
*//*
|
||||
* else if (auto * set = typeid_cast<ASTSetQuery *>(ast.get()))
|
||||
* {
|
||||
* for (auto & c : set->changes)
|
||||
* {
|
||||
* if (fuzz_rand() % 50 == 0)
|
||||
* {
|
||||
* c.value = fuzzField(c.value);
|
||||
* }
|
||||
* }
|
||||
* }
|
||||
*/
|
||||
else if (auto * set = typeid_cast<ASTSetQuery *>(ast.get()))
|
||||
{
|
||||
/// Fuzz settings
|
||||
for (auto & c : set->changes)
|
||||
if (fuzz_rand() % 50 == 0)
|
||||
c.value = fuzzField(c.value);
|
||||
}
|
||||
else if (auto * literal = typeid_cast<ASTLiteral *>(ast.get()))
|
||||
{
|
||||
// There is a caveat with fuzzing the children: many ASTs also keep the
|
||||
|
@ -810,7 +810,7 @@ ColumnPtr ColumnArray::filterTuple(const Filter & filt, ssize_t result_size_hint
|
||||
size_t tuple_size = tuple.tupleSize();
|
||||
|
||||
if (tuple_size == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
@ -1263,7 +1263,7 @@ ColumnPtr ColumnArray::replicateTuple(const Offsets & replicate_offsets) const
|
||||
size_t tuple_size = tuple.tupleSize();
|
||||
|
||||
if (tuple_size == 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty tuple");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty tuple");
|
||||
|
||||
Columns temporary_arrays(tuple_size);
|
||||
for (size_t i = 0; i < tuple_size; ++i)
|
||||
|
@ -1,7 +1,5 @@
|
||||
#include <Common/Arena.h>
|
||||
#include <Common/SipHash.h>
|
||||
#include <Common/NaNUtils.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Common/WeakHash.h>
|
||||
#include <Columns/ColumnDecimal.h>
|
||||
@ -28,7 +26,6 @@ namespace ErrorCodes
|
||||
{
|
||||
extern const int LOGICAL_ERROR;
|
||||
extern const int ILLEGAL_COLUMN;
|
||||
extern const int SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT;
|
||||
extern const int NOT_IMPLEMENTED;
|
||||
}
|
||||
|
||||
@ -829,8 +826,7 @@ void ColumnNullable::applyNullMap(const ColumnNullable & other)
|
||||
void ColumnNullable::checkConsistency() const
|
||||
{
|
||||
if (null_map->size() != getNestedColumn().size())
|
||||
throw Exception(ErrorCodes::SIZES_OF_NESTED_COLUMNS_ARE_INCONSISTENT,
|
||||
"Logical error: Sizes of nested column and null map of Nullable column are not equal");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Sizes of nested column and null map of Nullable column are not equal");
|
||||
}
|
||||
|
||||
ColumnPtr ColumnNullable::createWithOffsets(const IColumn::Offsets & offsets, const ColumnConst & column_with_default_value, size_t total_rows, size_t shift) const
|
||||
|
@ -21,7 +21,7 @@ static bool sameConstants(const IColumn & a, const IColumn & b)
|
||||
ColumnWithTypeAndName getLeastSuperColumn(const std::vector<const ColumnWithTypeAndName *> & columns)
|
||||
{
|
||||
if (columns.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no src columns for supercolumn");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No src columns for supercolumn");
|
||||
|
||||
ColumnWithTypeAndName result = *columns[0];
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <base/types.h>
|
||||
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
#include <cpuid.h>
|
||||
#endif
|
||||
|
||||
@ -11,10 +11,10 @@
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace Cpu
|
||||
namespace CPU
|
||||
{
|
||||
|
||||
#if (defined(__x86_64__) || defined(__i386__))
|
||||
#if (defined(__x86_64__))
|
||||
/// Our version is independent of -mxsave option, because we do dynamic dispatch.
|
||||
inline UInt64 our_xgetbv(UInt32 xcr) noexcept
|
||||
{
|
||||
@ -30,7 +30,7 @@ inline UInt64 our_xgetbv(UInt32 xcr) noexcept
|
||||
|
||||
inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept /// NOLINT
|
||||
{
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
__cpuid_count(op, sub_op, res[0], res[1], res[2], res[3]);
|
||||
return true;
|
||||
#else
|
||||
@ -45,7 +45,7 @@ inline bool cpuid(UInt32 op, UInt32 sub_op, UInt32 * res) noexcept /// NOLINT
|
||||
|
||||
inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT
|
||||
{
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
__cpuid(op, res[0], res[1], res[2], res[3]);
|
||||
return true;
|
||||
#else
|
||||
@ -98,7 +98,7 @@ inline bool cpuid(UInt32 op, UInt32 * res) noexcept /// NOLINT
|
||||
OP(AMXTILE) \
|
||||
OP(AMXINT8)
|
||||
|
||||
union CpuInfo
|
||||
union CPUInfo
|
||||
{
|
||||
UInt32 info[4];
|
||||
|
||||
@ -110,9 +110,9 @@ union CpuInfo
|
||||
UInt32 edx;
|
||||
} registers;
|
||||
|
||||
inline explicit CpuInfo(UInt32 op) noexcept { cpuid(op, info); }
|
||||
inline explicit CPUInfo(UInt32 op) noexcept { cpuid(op, info); }
|
||||
|
||||
inline CpuInfo(UInt32 op, UInt32 sub_op) noexcept { cpuid(op, sub_op, info); }
|
||||
inline CPUInfo(UInt32 op, UInt32 sub_op) noexcept { cpuid(op, sub_op, info); }
|
||||
};
|
||||
|
||||
#define DEF_NAME(X) inline bool have##X() noexcept;
|
||||
@ -121,77 +121,77 @@ union CpuInfo
|
||||
|
||||
bool haveRDTSCP() noexcept
|
||||
{
|
||||
return (CpuInfo(0x80000001).registers.edx >> 27) & 1u;
|
||||
return (CPUInfo(0x80000001).registers.edx >> 27) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSE() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.edx >> 25) & 1u;
|
||||
return (CPUInfo(0x1).registers.edx >> 25) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSE2() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.edx >> 26) & 1u;
|
||||
return (CPUInfo(0x1).registers.edx >> 26) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSE3() noexcept
|
||||
{
|
||||
return CpuInfo(0x1).registers.ecx & 1u;
|
||||
return CPUInfo(0x1).registers.ecx & 1u;
|
||||
}
|
||||
|
||||
bool havePCLMUL() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 1) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 1) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSSE3() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 9) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 9) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSE41() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 19) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 19) & 1u;
|
||||
}
|
||||
|
||||
bool haveSSE42() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 20) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 20) & 1u;
|
||||
}
|
||||
|
||||
bool haveF16C() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 29) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 29) & 1u;
|
||||
}
|
||||
|
||||
bool havePOPCNT() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 23) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 23) & 1u;
|
||||
}
|
||||
|
||||
bool haveAES() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 25) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 25) & 1u;
|
||||
}
|
||||
|
||||
bool haveXSAVE() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 26) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 26) & 1u;
|
||||
}
|
||||
|
||||
bool haveOSXSAVE() noexcept
|
||||
{
|
||||
return (CpuInfo(0x1).registers.ecx >> 27) & 1u;
|
||||
return (CPUInfo(0x1).registers.ecx >> 27) & 1u;
|
||||
}
|
||||
|
||||
bool haveAVX() noexcept
|
||||
{
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
// http://www.intel.com/content/dam/www/public/us/en/documents/manuals/64-ia-32-architectures-optimization-manual.pdf
|
||||
// https://bugs.chromium.org/p/chromium/issues/detail?id=375968
|
||||
return haveOSXSAVE() // implies haveXSAVE()
|
||||
&& (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS
|
||||
&& ((CpuInfo(0x1).registers.ecx >> 28) & 1u); // AVX bit
|
||||
&& ((CPUInfo(0x1).registers.ecx >> 28) & 1u); // AVX bit
|
||||
#else
|
||||
return false;
|
||||
#endif
|
||||
@ -199,33 +199,33 @@ bool haveAVX() noexcept
|
||||
|
||||
bool haveFMA() noexcept
|
||||
{
|
||||
return haveAVX() && ((CpuInfo(0x1).registers.ecx >> 12) & 1u);
|
||||
return haveAVX() && ((CPUInfo(0x1).registers.ecx >> 12) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX2() noexcept
|
||||
{
|
||||
return haveAVX() && ((CpuInfo(0x7, 0).registers.ebx >> 5) & 1u);
|
||||
return haveAVX() && ((CPUInfo(0x7, 0).registers.ebx >> 5) & 1u);
|
||||
}
|
||||
|
||||
bool haveBMI1() noexcept
|
||||
{
|
||||
return (CpuInfo(0x7, 0).registers.ebx >> 3) & 1u;
|
||||
return (CPUInfo(0x7, 0).registers.ebx >> 3) & 1u;
|
||||
}
|
||||
|
||||
bool haveBMI2() noexcept
|
||||
{
|
||||
return (CpuInfo(0x7, 0).registers.ebx >> 8) & 1u;
|
||||
return (CPUInfo(0x7, 0).registers.ebx >> 8) & 1u;
|
||||
}
|
||||
|
||||
bool haveAVX512F() noexcept
|
||||
{
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
// https://software.intel.com/en-us/articles/how-to-detect-knl-instruction-support
|
||||
return haveOSXSAVE() // implies haveXSAVE()
|
||||
&& (our_xgetbv(0) & 6u) == 6u // XMM state and YMM state are enabled by OS
|
||||
&& ((our_xgetbv(0) >> 5) & 7u) == 7u // ZMM state is enabled by OS
|
||||
&& CpuInfo(0x0).registers.eax >= 0x7 // leaf 7 is present
|
||||
&& ((CpuInfo(0x7, 0).registers.ebx >> 16) & 1u); // AVX512F bit
|
||||
&& CPUInfo(0x0).registers.eax >= 0x7 // leaf 7 is present
|
||||
&& ((CPUInfo(0x7, 0).registers.ebx >> 16) & 1u); // AVX512F bit
|
||||
#else
|
||||
return false;
|
||||
#endif
|
||||
@ -233,92 +233,92 @@ bool haveAVX512F() noexcept
|
||||
|
||||
bool haveAVX512DQ() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 17) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 17) & 1u);
|
||||
}
|
||||
|
||||
bool haveRDSEED() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 18) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 18) & 1u);
|
||||
}
|
||||
|
||||
bool haveADX() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 19) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 19) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512IFMA() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 21) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 21) & 1u);
|
||||
}
|
||||
|
||||
bool havePCOMMIT() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 22) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 22) & 1u);
|
||||
}
|
||||
|
||||
bool haveCLFLUSHOPT() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 23) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 23) & 1u);
|
||||
}
|
||||
|
||||
bool haveCLWB() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 24) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 24) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512PF() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 26) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 26) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512ER() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 27) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 27) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512CD() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 28) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 28) & 1u);
|
||||
}
|
||||
|
||||
bool haveSHA() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ebx >> 29) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ebx >> 29) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512BW() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 30) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 30) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512VL() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ebx >> 31) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ebx >> 31) & 1u);
|
||||
}
|
||||
|
||||
bool havePREFETCHWT1() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x7, 0).registers.ecx >> 0) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x7, 0).registers.ecx >> 0) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512VBMI() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 1) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ecx >> 1) & 1u);
|
||||
}
|
||||
|
||||
bool haveAVX512VBMI2() noexcept
|
||||
{
|
||||
return haveAVX512F() && ((CpuInfo(0x7, 0).registers.ecx >> 6) & 1u);
|
||||
return haveAVX512F() && ((CPUInfo(0x7, 0).registers.ecx >> 6) & 1u);
|
||||
}
|
||||
|
||||
bool haveRDRAND() noexcept
|
||||
{
|
||||
return CpuInfo(0x0).registers.eax >= 0x7 && ((CpuInfo(0x1).registers.ecx >> 30) & 1u);
|
||||
return CPUInfo(0x0).registers.eax >= 0x7 && ((CPUInfo(0x1).registers.ecx >> 30) & 1u);
|
||||
}
|
||||
|
||||
inline bool haveAMX() noexcept
|
||||
{
|
||||
#if defined(__x86_64__) || defined(__i386__)
|
||||
#if defined(__x86_64__)
|
||||
// http://www.intel.com/content/dam/www/public/us/en/documents/manuals/64-ia-32-architectures-optimization-manual.pdf
|
||||
return haveOSXSAVE() // implies haveXSAVE()
|
||||
&& ((our_xgetbv(0) >> 17) & 0x3) == 0x3; // AMX state are enabled by OS
|
||||
@ -330,22 +330,22 @@ inline bool haveAMX() noexcept
|
||||
bool haveAMXBF16() noexcept
|
||||
{
|
||||
return haveAMX()
|
||||
&& ((CpuInfo(0x7, 0).registers.edx >> 22) & 1u); // AMX-BF16 bit
|
||||
&& ((CPUInfo(0x7, 0).registers.edx >> 22) & 1u); // AMX-BF16 bit
|
||||
}
|
||||
|
||||
bool haveAMXTILE() noexcept
|
||||
{
|
||||
return haveAMX()
|
||||
&& ((CpuInfo(0x7, 0).registers.edx >> 24) & 1u); // AMX-TILE bit
|
||||
&& ((CPUInfo(0x7, 0).registers.edx >> 24) & 1u); // AMX-TILE bit
|
||||
}
|
||||
|
||||
bool haveAMXINT8() noexcept
|
||||
{
|
||||
return haveAMX()
|
||||
&& ((CpuInfo(0x7, 0).registers.edx >> 25) & 1u); // AMX-INT8 bit
|
||||
&& ((CPUInfo(0x7, 0).registers.edx >> 25) & 1u); // AMX-INT8 bit
|
||||
}
|
||||
|
||||
struct CpuFlagsCache
|
||||
struct CPUFlagsCache
|
||||
{
|
||||
#define DEF_NAME(X) static inline bool have_##X = have##X();
|
||||
CPU_ID_ENUMERATE(DEF_NAME)
|
||||
@ -354,4 +354,3 @@ struct CpuFlagsCache
|
||||
|
||||
}
|
||||
}
|
||||
|
@ -262,6 +262,9 @@
|
||||
M(ActiveTimersInQueryProfiler, "Number of Active thread local timers in QueryProfiler") \
|
||||
M(RefreshableViews, "Number materialized views with periodic refreshing (REFRESH)") \
|
||||
M(RefreshingViews, "Number of materialized views currently executing a refresh") \
|
||||
M(StorageBufferFlushThreads, "Number of threads for background flushes in StorageBuffer") \
|
||||
M(StorageBufferFlushThreadsActive, "Number of threads for background flushes in StorageBuffer running a task") \
|
||||
M(StorageBufferFlushThreadsScheduled, "Number of queued or active threads for background flushes in StorageBuffer")
|
||||
|
||||
#ifdef APPLY_FOR_EXTERNAL_METRICS
|
||||
#define APPLY_FOR_METRICS(M) APPLY_FOR_BUILTIN_METRICS(M) APPLY_FOR_EXTERNAL_METRICS(M)
|
||||
|
@ -2067,8 +2067,8 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
|
||||
if (opcode != 0)
|
||||
{ // standard opcode
|
||||
// Only interpret opcodes that are recognized by the version we're parsing;
|
||||
// the others are vendor extensions and we should ignore them.
|
||||
switch (opcode) // NOLINT(bugprone-switch-missing-default-case)
|
||||
// the others are vendor extensions, and we should ignore them.
|
||||
switch (opcode)
|
||||
{
|
||||
case DW_LNS_copy:
|
||||
basicBlock_ = false;
|
||||
@ -2121,6 +2121,7 @@ Dwarf::LineNumberVM::StepResult Dwarf::LineNumberVM::step(std::string_view & pro
|
||||
}
|
||||
isa_ = readULEB(program);
|
||||
return CONTINUE;
|
||||
default:
|
||||
}
|
||||
|
||||
// Unrecognized standard opcode, slurp the appropriate number of LEB
|
||||
|
@ -6,7 +6,6 @@
|
||||
#include <chrono>
|
||||
#include <condition_variable>
|
||||
#include <mutex>
|
||||
#include <optional>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -44,7 +43,8 @@ static struct InitFiu
|
||||
REGULAR(dummy_failpoint) \
|
||||
REGULAR(prefetched_reader_pool_failpoint) \
|
||||
PAUSEABLE_ONCE(dummy_pausable_failpoint_once) \
|
||||
PAUSEABLE(dummy_pausable_failpoint)
|
||||
PAUSEABLE(dummy_pausable_failpoint) \
|
||||
ONCE(execute_query_calling_empty_set_result_func_on_exception)
|
||||
|
||||
namespace FailPoints
|
||||
{
|
||||
|
@ -18,7 +18,6 @@
|
||||
#pragma clang diagnostic pop
|
||||
#endif
|
||||
|
||||
#include <any>
|
||||
#include <unordered_map>
|
||||
|
||||
namespace DB
|
||||
|
@ -17,7 +17,7 @@ private:
|
||||
template <typename T> friend class FiberLocal;
|
||||
|
||||
public:
|
||||
template< typename StackAlloc, typename Fn>
|
||||
template <typename StackAlloc, typename Fn>
|
||||
Fiber(StackAlloc && salloc, Fn && fn) : impl(std::allocator_arg_t(), std::forward<StackAlloc>(salloc), RoutineImpl(std::forward<Fn>(fn)))
|
||||
{
|
||||
}
|
||||
@ -46,6 +46,12 @@ public:
|
||||
current_fiber = parent_fiber;
|
||||
}
|
||||
|
||||
static FiberPtr & getCurrentFiber()
|
||||
{
|
||||
thread_local static FiberPtr current_fiber;
|
||||
return current_fiber;
|
||||
}
|
||||
|
||||
private:
|
||||
template <typename Fn>
|
||||
struct RoutineImpl
|
||||
@ -74,12 +80,6 @@ private:
|
||||
Fn fn;
|
||||
};
|
||||
|
||||
static FiberPtr & getCurrentFiber()
|
||||
{
|
||||
thread_local static FiberPtr current_fiber;
|
||||
return current_fiber;
|
||||
}
|
||||
|
||||
/// Special wrapper to store data in uniquer_ptr.
|
||||
struct DataWrapper
|
||||
{
|
||||
@ -146,4 +146,3 @@ private:
|
||||
|
||||
T main_instance;
|
||||
};
|
||||
|
||||
|
@ -91,7 +91,7 @@ public:
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
return x.getValue().template convertTo<T>() / x.getScaleMultiplier().template convertTo<T>();
|
||||
else
|
||||
return (x.getValue() / x.getScaleMultiplier()). template convertTo<T>();
|
||||
return (x.getValue() / x.getScaleMultiplier()).template convertTo<T>();
|
||||
}
|
||||
|
||||
T operator() (const AggregateFunctionStateData &) const
|
||||
|
@ -130,13 +130,12 @@ LogSeriesLimiter::LogSeriesLimiter(LoggerPtr logger_, size_t allowed_count_, tim
|
||||
if (last_time + interval_s_ <= now)
|
||||
{
|
||||
debug_message = fmt::format(
|
||||
" (LogSeriesLimiter: on interval from {} to {} accepted series {} / {} for the logger {} : {})",
|
||||
" (LogSeriesLimiter: on interval from {} to {} accepted series {} / {} for the logger {})",
|
||||
DateLUT::instance().timeToString(last_time),
|
||||
DateLUT::instance().timeToString(now),
|
||||
accepted_count,
|
||||
total_count,
|
||||
logger->name(),
|
||||
double(name_hash));
|
||||
logger->name());
|
||||
|
||||
register_as_first();
|
||||
return;
|
||||
|
@ -312,7 +312,7 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(ParallelReplicasStealingLeftoversMicroseconds, "Time spent collecting orphaned segments") \
|
||||
M(ParallelReplicasCollectingOwnedSegmentsMicroseconds, "Time spent collecting segments meant by hash") \
|
||||
\
|
||||
M(PerfCpuCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \
|
||||
M(PerfCPUCycles, "Total cycles. Be wary of what happens during CPU frequency scaling.") \
|
||||
M(PerfInstructions, "Retired instructions. Be careful, these can be affected by various issues, most notably hardware interrupt counts.") \
|
||||
M(PerfCacheReferences, "Cache accesses. Usually, this indicates Last Level Cache accesses, but this may vary depending on your CPU. This may include prefetches and coherency messages; again this depends on the design of your CPU.") \
|
||||
M(PerfCacheMisses, "Cache misses. Usually this indicates Last Level Cache misses; this is intended to be used in conjunction with the PERFCOUNTHWCACHEREFERENCES event to calculate cache miss rates.") \
|
||||
@ -321,12 +321,12 @@ The server successfully detected this situation and will download merged part fr
|
||||
M(PerfBusCycles, "Bus cycles, which can be different from total cycles.") \
|
||||
M(PerfStalledCyclesFrontend, "Stalled cycles during issue.") \
|
||||
M(PerfStalledCyclesBackend, "Stalled cycles during retirement.") \
|
||||
M(PerfRefCpuCycles, "Total cycles; not affected by CPU frequency scaling.") \
|
||||
M(PerfRefCPUCycles, "Total cycles; not affected by CPU frequency scaling.") \
|
||||
\
|
||||
M(PerfCpuClock, "The CPU clock, a high-resolution per-CPU timer") \
|
||||
M(PerfCPUClock, "The CPU clock, a high-resolution per-CPU timer") \
|
||||
M(PerfTaskClock, "A clock count specific to the task that is running") \
|
||||
M(PerfContextSwitches, "Number of context switches") \
|
||||
M(PerfCpuMigrations, "Number of times the process has migrated to a new CPU") \
|
||||
M(PerfCPUMigrations, "Number of times the process has migrated to a new CPU") \
|
||||
M(PerfAlignmentFaults, "Number of alignment faults. These happen when unaligned memory accesses happen; the kernel can handle these but it reduces performance. This happens only on some architectures (never on x86).") \
|
||||
M(PerfEmulationFaults, "Number of emulation faults. The kernel sometimes traps on unimplemented instructions and emulates them for user space. This can negatively impact performance.") \
|
||||
M(PerfMinEnabledTime, "For all events, minimum time that an event was enabled. Used to track event multiplexing influence") \
|
||||
|
@ -91,7 +91,7 @@ void SensitiveDataMasker::setInstance(std::unique_ptr<SensitiveDataMasker>&& sen
|
||||
{
|
||||
|
||||
if (!sensitive_data_masker_)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: the 'sensitive_data_masker' is not set");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "The 'sensitive_data_masker' is not set");
|
||||
|
||||
if (sensitive_data_masker_->rulesCount() > 0)
|
||||
{
|
||||
|
@ -209,7 +209,7 @@ public:
|
||||
{
|
||||
if (!is_reference_128)
|
||||
throw DB::Exception(
|
||||
DB::ErrorCodes::LOGICAL_ERROR, "Logical error: can't call get128Reference when is_reference_128 is not set");
|
||||
DB::ErrorCodes::LOGICAL_ERROR, "Can't call get128Reference when is_reference_128 is not set");
|
||||
finalize();
|
||||
const auto lo = v0 ^ v1 ^ v2 ^ v3;
|
||||
v1 ^= 0xdd;
|
||||
|
@ -448,9 +448,6 @@ toStringEveryLineImpl([[maybe_unused]] bool fatal, const StackTraceRefTriple & s
|
||||
DB::writePointerHex(frame.physical_addr, out);
|
||||
}
|
||||
|
||||
if (frame.object.has_value())
|
||||
out << " in " << *frame.object;
|
||||
|
||||
callback(out.str());
|
||||
};
|
||||
#else
|
||||
|
@ -1,7 +1,7 @@
|
||||
#include <base/defines.h>
|
||||
#include <Common/TargetSpecific.h>
|
||||
|
||||
#include <Common/CpuId.h>
|
||||
#include <Common/CPUID.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -9,25 +9,25 @@ namespace DB
|
||||
UInt32 getSupportedArchs()
|
||||
{
|
||||
UInt32 result = 0;
|
||||
if (Cpu::CpuFlagsCache::have_SSE42)
|
||||
if (CPU::CPUFlagsCache::have_SSE42)
|
||||
result |= static_cast<UInt32>(TargetArch::SSE42);
|
||||
if (Cpu::CpuFlagsCache::have_AVX)
|
||||
if (CPU::CPUFlagsCache::have_AVX)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX);
|
||||
if (Cpu::CpuFlagsCache::have_AVX2)
|
||||
if (CPU::CPUFlagsCache::have_AVX2)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX2);
|
||||
if (Cpu::CpuFlagsCache::have_AVX512F)
|
||||
if (CPU::CPUFlagsCache::have_AVX512F)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX512F);
|
||||
if (Cpu::CpuFlagsCache::have_AVX512BW)
|
||||
if (CPU::CPUFlagsCache::have_AVX512BW)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX512BW);
|
||||
if (Cpu::CpuFlagsCache::have_AVX512VBMI)
|
||||
if (CPU::CPUFlagsCache::have_AVX512VBMI)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX512VBMI);
|
||||
if (Cpu::CpuFlagsCache::have_AVX512VBMI2)
|
||||
if (CPU::CPUFlagsCache::have_AVX512VBMI2)
|
||||
result |= static_cast<UInt32>(TargetArch::AVX512VBMI2);
|
||||
if (Cpu::CpuFlagsCache::have_AMXBF16)
|
||||
if (CPU::CPUFlagsCache::have_AMXBF16)
|
||||
result |= static_cast<UInt32>(TargetArch::AMXBF16);
|
||||
if (Cpu::CpuFlagsCache::have_AMXTILE)
|
||||
if (CPU::CPUFlagsCache::have_AMXTILE)
|
||||
result |= static_cast<UInt32>(TargetArch::AMXTILE);
|
||||
if (Cpu::CpuFlagsCache::have_AMXINT8)
|
||||
if (CPU::CPUFlagsCache::have_AMXINT8)
|
||||
result |= static_cast<UInt32>(TargetArch::AMXINT8);
|
||||
return result;
|
||||
}
|
||||
|
@ -6,10 +6,8 @@
|
||||
#include "ProcfsMetricsProvider.h"
|
||||
#include "hasLinuxCapability.h"
|
||||
|
||||
#include <filesystem>
|
||||
#include <fstream>
|
||||
#include <optional>
|
||||
#include <unordered_set>
|
||||
|
||||
#include <fcntl.h>
|
||||
#include <unistd.h>
|
||||
@ -36,7 +34,7 @@ namespace ProfileEvents
|
||||
extern const Event OSReadBytes;
|
||||
extern const Event OSWriteBytes;
|
||||
|
||||
extern const Event PerfCpuCycles;
|
||||
extern const Event PerfCPUCycles;
|
||||
extern const Event PerfInstructions;
|
||||
extern const Event PerfCacheReferences;
|
||||
extern const Event PerfCacheMisses;
|
||||
@ -45,12 +43,12 @@ namespace ProfileEvents
|
||||
extern const Event PerfBusCycles;
|
||||
extern const Event PerfStalledCyclesFrontend;
|
||||
extern const Event PerfStalledCyclesBackend;
|
||||
extern const Event PerfRefCpuCycles;
|
||||
extern const Event PerfRefCPUCycles;
|
||||
|
||||
extern const Event PerfCpuClock;
|
||||
extern const Event PerfCPUClock;
|
||||
extern const Event PerfTaskClock;
|
||||
extern const Event PerfContextSwitches;
|
||||
extern const Event PerfCpuMigrations;
|
||||
extern const Event PerfCPUMigrations;
|
||||
extern const Event PerfAlignmentFaults;
|
||||
extern const Event PerfEmulationFaults;
|
||||
extern const Event PerfMinEnabledTime;
|
||||
@ -218,7 +216,7 @@ thread_local PerfEventsCounters current_thread_counters;
|
||||
|
||||
// descriptions' source: http://man7.org/linux/man-pages/man2/perf_event_open.2.html
|
||||
static const PerfEventInfo raw_events_info[] = {
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CPU_CYCLES, PerfCpuCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CPU_CYCLES, PerfCPUCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_INSTRUCTIONS, PerfInstructions),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CACHE_REFERENCES, PerfCacheReferences),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_CACHE_MISSES, PerfCacheMisses),
|
||||
@ -227,13 +225,13 @@ static const PerfEventInfo raw_events_info[] = {
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_BUS_CYCLES, PerfBusCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_FRONTEND, PerfStalledCyclesFrontend),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_STALLED_CYCLES_BACKEND, PerfStalledCyclesBackend),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_REF_CPU_CYCLES, PerfRefCpuCycles),
|
||||
HARDWARE_EVENT(PERF_COUNT_HW_REF_CPU_CYCLES, PerfRefCPUCycles),
|
||||
|
||||
// `cpu-clock` is a bit broken according to this: https://stackoverflow.com/a/56967896
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_CLOCK, PerfCpuClock),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_CLOCK, PerfCPUClock),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_TASK_CLOCK, PerfTaskClock),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CONTEXT_SWITCHES, PerfContextSwitches),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_MIGRATIONS, PerfCpuMigrations),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_CPU_MIGRATIONS, PerfCPUMigrations),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_ALIGNMENT_FAULTS, PerfAlignmentFaults),
|
||||
SOFTWARE_EVENT(PERF_COUNT_SW_EMULATION_FAULTS, PerfEmulationFaults),
|
||||
|
||||
|
@ -1,8 +1,8 @@
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/defines.h> /// THREAD_SANITIZER
|
||||
#include <Common/checkStackSize.h>
|
||||
#include <Common/Exception.h>
|
||||
#include <base/getThreadId.h>
|
||||
#include <base/scope_guard.h>
|
||||
#include <base/defines.h> /// THREAD_SANITIZER
|
||||
#include <Common/Fiber.h>
|
||||
#include <sys/resource.h>
|
||||
#include <pthread.h>
|
||||
#include <unistd.h>
|
||||
@ -114,6 +114,10 @@ __attribute__((__weak__)) void checkStackSize()
|
||||
{
|
||||
using namespace DB;
|
||||
|
||||
/// Not implemented for coroutines.
|
||||
if (Fiber::getCurrentFiber())
|
||||
return;
|
||||
|
||||
if (!stack_address)
|
||||
max_stack_size = getStackSize(&stack_address);
|
||||
|
||||
@ -136,7 +140,7 @@ __attribute__((__weak__)) void checkStackSize()
|
||||
|
||||
/// We assume that stack grows towards lower addresses. And that it starts to grow from the end of a chunk of memory of max_stack_size.
|
||||
if (int_frame_address > int_stack_address + max_stack_size)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: frame address is greater than stack begin address");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Frame address is greater than stack begin address");
|
||||
|
||||
size_t stack_size = int_stack_address + max_stack_size - int_frame_address;
|
||||
size_t max_stack_size_allowed = static_cast<size_t>(max_stack_size * STACK_SIZE_FREE_RATIO);
|
||||
|
@ -91,6 +91,7 @@ enum class MagicNumber : uint8_t
|
||||
Decimal32 = 19,
|
||||
Decimal64 = 20,
|
||||
IPv4 = 21,
|
||||
Date32 = 22,
|
||||
};
|
||||
|
||||
MagicNumber serializeTypeId(std::optional<TypeIndex> type_id)
|
||||
@ -109,6 +110,7 @@ MagicNumber serializeTypeId(std::optional<TypeIndex> type_id)
|
||||
case TypeIndex::Int32: return MagicNumber::Int32;
|
||||
case TypeIndex::Int64: return MagicNumber::Int64;
|
||||
case TypeIndex::Date: return MagicNumber::Date;
|
||||
case TypeIndex::Date32: return MagicNumber::Date32;
|
||||
case TypeIndex::DateTime: return MagicNumber::DateTime;
|
||||
case TypeIndex::DateTime64: return MagicNumber::DateTime64;
|
||||
case TypeIndex::Enum8: return MagicNumber::Enum8;
|
||||
@ -137,6 +139,7 @@ TypeIndex deserializeTypeId(uint8_t serialized_type_id)
|
||||
case MagicNumber::Int32: return TypeIndex::Int32;
|
||||
case MagicNumber::Int64: return TypeIndex::Int64;
|
||||
case MagicNumber::Date: return TypeIndex::Date;
|
||||
case MagicNumber::Date32: return TypeIndex::Date32;
|
||||
case MagicNumber::DateTime: return TypeIndex::DateTime;
|
||||
case MagicNumber::DateTime64: return TypeIndex::DateTime64;
|
||||
case MagicNumber::Enum8: return TypeIndex::Enum8;
|
||||
@ -165,6 +168,7 @@ TypeIndex baseType(TypeIndex type_idx)
|
||||
return TypeIndex::Int16;
|
||||
case TypeIndex::Int32:
|
||||
case TypeIndex::Decimal32:
|
||||
case TypeIndex::Date32:
|
||||
return TypeIndex::Int32;
|
||||
case TypeIndex::Int64:
|
||||
case TypeIndex::Decimal64:
|
||||
@ -205,6 +209,7 @@ TypeIndex typeIdx(const IDataType * data_type)
|
||||
case TypeIndex::UInt16:
|
||||
case TypeIndex::Enum16:
|
||||
case TypeIndex::Date:
|
||||
case TypeIndex::Date32:
|
||||
case TypeIndex::Int32:
|
||||
case TypeIndex::UInt32:
|
||||
case TypeIndex::IPv4:
|
||||
|
@ -85,7 +85,7 @@
|
||||
M(OSReadChars) \
|
||||
M(OSWriteChars) \
|
||||
\
|
||||
M(PerfCpuCycles) \
|
||||
M(PerfCPUCycles) \
|
||||
M(PerfInstructions) \
|
||||
M(PerfCacheReferences) \
|
||||
M(PerfCacheMisses) \
|
||||
@ -94,12 +94,12 @@
|
||||
M(PerfBusCycles) \
|
||||
M(PerfStalledCyclesFrontend) \
|
||||
M(PerfStalledCyclesBackend) \
|
||||
M(PerfRefCpuCycles) \
|
||||
M(PerfRefCPUCycles) \
|
||||
\
|
||||
M(PerfCpuClock) \
|
||||
M(PerfCPUClock) \
|
||||
M(PerfTaskClock) \
|
||||
M(PerfContextSwitches) \
|
||||
M(PerfCpuMigrations) \
|
||||
M(PerfCPUMigrations) \
|
||||
M(PerfAlignmentFaults) \
|
||||
M(PerfEmulationFaults) \
|
||||
M(PerfMinEnabledTime) \
|
||||
|
@ -1583,7 +1583,7 @@ struct KeeperStorageListRequestProcessor final : public KeeperStorageRequestProc
|
||||
{
|
||||
auto path_prefix = request.path;
|
||||
if (path_prefix.empty())
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: path cannot be empty");
|
||||
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Path cannot be empty");
|
||||
|
||||
const auto & children = node_it->value.getChildren();
|
||||
response.names.reserve(children.size());
|
||||
|
@ -152,7 +152,7 @@ bool notEqualsOp(A a, B b)
|
||||
}
|
||||
|
||||
/// Converts numeric to an equal numeric of other type.
|
||||
/// When `strict` is `true` check that result exactly same as input, otherwise just check overflow
|
||||
/// When `strict` is `true` check that result exactly the same as input, otherwise just check overflow
|
||||
template <typename From, typename To, bool strict = true>
|
||||
inline bool NO_SANITIZE_UNDEFINED convertNumeric(From value, To & result)
|
||||
{
|
||||
|
@ -40,7 +40,7 @@ bool PacketEndpoint::tryReceivePacket(IMySQLReadPacket & packet, UInt64 millisec
|
||||
ReadBufferFromPocoSocket * socket_in = typeid_cast<ReadBufferFromPocoSocket *>(in);
|
||||
|
||||
if (!socket_in)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: Attempt to pull the duration in a non socket stream");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Attempt to pull the duration in a non socket stream");
|
||||
|
||||
if (!socket_in->poll(millisecond * 1000))
|
||||
return false;
|
||||
|
@ -1019,6 +1019,7 @@ class IColumn;
|
||||
M(Bool, input_format_try_infer_integers, true, "Try to infer integers instead of floats while schema inference in text formats", 0) \
|
||||
M(Bool, input_format_try_infer_dates, true, "Try to infer dates from string fields while schema inference in text formats", 0) \
|
||||
M(Bool, input_format_try_infer_datetimes, true, "Try to infer datetimes from string fields while schema inference in text formats", 0) \
|
||||
M(Bool, input_format_try_infer_exponent_floats, false, "Try to infer floats in exponential notation while schema inference in text formats", 0) \
|
||||
M(Bool, output_format_markdown_escape_special_characters, false, "Escape special characters in Markdown", 0) \
|
||||
M(Bool, input_format_protobuf_flatten_google_wrappers, false, "Enable Google wrappers for regular non-nested columns, e.g. google.protobuf.StringValue 'str' for String column 'str'. For Nullable columns empty wrappers are recognized as defaults, and missing as nulls", 0) \
|
||||
M(Bool, output_format_protobuf_nullables_with_google_wrappers, false, "When serializing Nullable columns with Google wrappers, serialize default values as empty wrappers. If turned off, default and null values are not serialized", 0) \
|
||||
@ -1146,6 +1147,8 @@ class IColumn;
|
||||
M(Bool, output_format_sql_insert_use_replace, false, "Use REPLACE statement instead of INSERT", 0) \
|
||||
M(Bool, output_format_sql_insert_quote_names, true, "Quote column names with '`' characters", 0) \
|
||||
\
|
||||
M(Bool, output_format_values_escape_quote_with_quote, false, "If true escape ' with '', otherwise quoted with \\'", 0) \
|
||||
\
|
||||
M(Bool, output_format_bson_string_as_string, false, "Use BSON String type instead of Binary for String columns.", 0) \
|
||||
M(Bool, input_format_bson_skip_fields_with_unsupported_types_in_schema_inference, false, "Skip fields with unsupported types while schema inference for format BSON.", 0) \
|
||||
\
|
||||
|
@ -85,6 +85,7 @@ namespace SettingsChangesHistory
|
||||
static std::map<ClickHouseVersion, SettingsChangesHistory::SettingsChanges> settings_changes_history =
|
||||
{
|
||||
{"24.2", {
|
||||
{"input_format_try_infer_exponent_floats", true, false, "Don't infer floats in exponential notation by default"},
|
||||
{"async_insert_max_data_size", 1000000, 10485760, "The previous value appeared to be too small."},
|
||||
{"async_insert_poll_timeout_ms", 10, 10, "Timeout in milliseconds for polling data from asynchronous insert queue"},
|
||||
{"async_insert_use_adaptive_busy_timeout", true, true, "Use adaptive asynchronous insert timeout"},
|
||||
|
@ -1,8 +1,7 @@
|
||||
#include <Core/SettingsFields.h>
|
||||
|
||||
#include <Core/Field.h>
|
||||
#include <Core/AccurateComparison.h>
|
||||
#include <Common/getNumberOfPhysicalCPUCores.h>
|
||||
#include <Common/FieldVisitorConvertToNumber.h>
|
||||
#include <Common/logger_useful.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeString.h>
|
||||
@ -13,6 +12,7 @@
|
||||
|
||||
#include <cmath>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
namespace ErrorCodes
|
||||
@ -20,6 +20,7 @@ namespace ErrorCodes
|
||||
extern const int SIZE_OF_FIXED_STRING_DOESNT_MATCH;
|
||||
extern const int CANNOT_PARSE_BOOL;
|
||||
extern const int CANNOT_PARSE_NUMBER;
|
||||
extern const int CANNOT_CONVERT_TYPE;
|
||||
}
|
||||
|
||||
|
||||
@ -48,9 +49,51 @@ namespace
|
||||
T fieldToNumber(const Field & f)
|
||||
{
|
||||
if (f.getType() == Field::Types::String)
|
||||
{
|
||||
return stringToNumber<T>(f.get<const String &>());
|
||||
}
|
||||
else if (f.getType() == Field::Types::UInt64)
|
||||
{
|
||||
T result;
|
||||
if (!accurate::convertNumeric(f.get<UInt64>(), result))
|
||||
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
|
||||
return result;
|
||||
}
|
||||
else if (f.getType() == Field::Types::Int64)
|
||||
{
|
||||
T result;
|
||||
if (!accurate::convertNumeric(f.get<Int64>(), result))
|
||||
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Field value {} is out of range of {} type", f, demangle(typeid(T).name()));
|
||||
return result;
|
||||
}
|
||||
else if (f.getType() == Field::Types::Bool)
|
||||
{
|
||||
return T(f.get<bool>());
|
||||
}
|
||||
else if (f.getType() == Field::Types::Float64)
|
||||
{
|
||||
Float64 x = f.get<Float64>();
|
||||
if constexpr (std::is_floating_point_v<T>)
|
||||
{
|
||||
return T(x);
|
||||
}
|
||||
else
|
||||
{
|
||||
if (!isFinite(x))
|
||||
{
|
||||
/// Conversion of infinite values to integer is undefined.
|
||||
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert infinite value to integer type");
|
||||
}
|
||||
else if (x > Float64(std::numeric_limits<T>::max()) || x < Float64(std::numeric_limits<T>::lowest()))
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Cannot convert out of range floating point value to integer type");
|
||||
}
|
||||
else
|
||||
return T(x);
|
||||
}
|
||||
}
|
||||
else
|
||||
return applyVisitor(FieldVisitorConvertToNumber<T>(), f);
|
||||
throw Exception(ErrorCodes::CANNOT_CONVERT_TYPE, "Invalid value {} of the setting, which needs {}", f, demangle(typeid(T).name()));
|
||||
}
|
||||
|
||||
Map stringToMap(const String & str)
|
||||
@ -174,7 +217,7 @@ namespace
|
||||
if (f.getType() == Field::Types::String)
|
||||
return stringToMaxThreads(f.get<const String &>());
|
||||
else
|
||||
return applyVisitor(FieldVisitorConvertToNumber<UInt64>(), f);
|
||||
return fieldToNumber<UInt64>(f);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -239,7 +239,7 @@ static DataTypePtr create(const ASTPtr & arguments)
|
||||
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
|
||||
|
||||
if (function_name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed");
|
||||
|
||||
AggregateFunctionProperties properties;
|
||||
AggregateFunctionPtr function = AggregateFunctionFactory::instance().get(function_name, action, argument_types, params_row, properties);
|
||||
|
@ -141,7 +141,7 @@ static std::pair<DataTypePtr, DataTypeCustomDescPtr> create(const ASTPtr & argum
|
||||
argument_types.push_back(DataTypeFactory::instance().get(arguments->children[i]));
|
||||
|
||||
if (function_name.empty())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: empty name of aggregate function passed");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Empty name of aggregate function passed");
|
||||
|
||||
AggregateFunctionProperties properties;
|
||||
/// NullsAction is not part of the type definition, instead it will have transformed the function into a different one
|
||||
|
@ -334,9 +334,12 @@ bool SerializationString::tryDeserializeTextEscaped(IColumn & column, ReadBuffer
|
||||
return read<bool>(column, [&](ColumnString::Chars & data) { readEscapedStringInto(data, istr); return true; });
|
||||
}
|
||||
|
||||
void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings &) const
|
||||
void SerializationString::serializeTextQuoted(const IColumn & column, size_t row_num, WriteBuffer & ostr, const FormatSettings & settings) const
|
||||
{
|
||||
writeQuotedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
if (settings.values.escape_quote_with_quote)
|
||||
writeQuotedStringPostgreSQL(assert_cast<const ColumnString &>(column).getDataAt(row_num).toView(), ostr);
|
||||
else
|
||||
writeQuotedString(assert_cast<const ColumnString &>(column).getDataAt(row_num), ostr);
|
||||
}
|
||||
|
||||
|
||||
|
@ -779,7 +779,7 @@ static void writeFieldsToColumn(
|
||||
casted_int32_column->insertValue(num & 0x800000 ? num | 0xFF000000 : num);
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: it is a bug.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug.");
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -844,7 +844,7 @@ static inline bool differenceSortingKeys(const Tuple & row_old_data, const Tuple
|
||||
static inline size_t onUpdateData(const Row & rows_data, Block & buffer, size_t version, const std::vector<size_t> & sorting_columns_index)
|
||||
{
|
||||
if (rows_data.size() % 2 != 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug.");
|
||||
|
||||
size_t prev_bytes = buffer.bytes();
|
||||
std::vector<bool> writeable_rows_mask(rows_data.size());
|
||||
|
@ -227,7 +227,7 @@ void parseMatchNode(UInt64 parent_id, UInt64 & id, const YAML::Node & node, Resu
|
||||
|
||||
if (!match.contains(key_name))
|
||||
{
|
||||
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "Yaml match rule must contain key {}", key_name);
|
||||
throw Exception(ErrorCodes::INVALID_CONFIG_PARAMETER, "YAML match rule must contain key {}", key_name);
|
||||
}
|
||||
for (const auto & [key, node_] : match)
|
||||
{
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <memory>
|
||||
#include <IO/AsynchronousReader.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <IO/AsynchronousReader.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Common/ThreadPool_fwd.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
@ -22,7 +22,7 @@
|
||||
#include <Core/Types.h>
|
||||
#include <Disks/DirectoryIterator.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Common/Exception.h>
|
||||
#include "config.h"
|
||||
|
||||
|
@ -2,7 +2,7 @@
|
||||
|
||||
#include <Disks/ObjectStorages/ObjectStorageIterator.h>
|
||||
#include <Common/ThreadPool.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <mutex>
|
||||
#include <Common/CurrentMetrics.h>
|
||||
|
||||
|
@ -13,7 +13,7 @@
|
||||
#include <IO/S3/getObjectInfo.h>
|
||||
#include <IO/S3/copyS3File.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
|
||||
#include <Disks/ObjectStorages/S3/diskSettings.h>
|
||||
|
@ -181,6 +181,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
|
||||
format_settings.values.allow_data_after_semicolon = settings.input_format_values_allow_data_after_semicolon;
|
||||
format_settings.values.deduce_templates_of_expressions = settings.input_format_values_deduce_templates_of_expressions;
|
||||
format_settings.values.interpret_expressions = settings.input_format_values_interpret_expressions;
|
||||
format_settings.values.escape_quote_with_quote = settings.output_format_values_escape_quote_with_quote;
|
||||
format_settings.with_names_use_header = settings.input_format_with_names_use_header;
|
||||
format_settings.with_types_use_header = settings.input_format_with_types_use_header;
|
||||
format_settings.write_statistics = settings.output_format_write_statistics;
|
||||
@ -228,6 +229,7 @@ FormatSettings getFormatSettings(const ContextPtr & context, const Settings & se
|
||||
format_settings.try_infer_integers = settings.input_format_try_infer_integers;
|
||||
format_settings.try_infer_dates = settings.input_format_try_infer_dates;
|
||||
format_settings.try_infer_datetimes = settings.input_format_try_infer_datetimes;
|
||||
format_settings.try_infer_exponent_floats = settings.input_format_try_infer_exponent_floats;
|
||||
format_settings.markdown.escape_special_characters = settings.output_format_markdown_escape_special_characters;
|
||||
format_settings.bson.output_string_as_string = settings.output_format_bson_string_as_string;
|
||||
format_settings.bson.skip_fields_with_unsupported_types_in_schema_inference = settings.input_format_bson_skip_fields_with_unsupported_types_in_schema_inference;
|
||||
|
@ -46,6 +46,7 @@ struct FormatSettings
|
||||
bool try_infer_integers = false;
|
||||
bool try_infer_dates = false;
|
||||
bool try_infer_datetimes = false;
|
||||
bool try_infer_exponent_floats = false;
|
||||
|
||||
enum class DateTimeInputFormat
|
||||
{
|
||||
@ -361,6 +362,7 @@ struct FormatSettings
|
||||
bool deduce_templates_of_expressions = true;
|
||||
bool accurate_types_of_literals = true;
|
||||
bool allow_data_after_semicolon = false;
|
||||
bool escape_quote_with_quote = false;
|
||||
} values;
|
||||
|
||||
enum class ORCCompression
|
||||
|
@ -17,6 +17,7 @@
|
||||
#include <IO/ReadHelpers.h>
|
||||
#include <IO/parseDateTimeBestEffort.h>
|
||||
#include <IO/PeekableReadBuffer.h>
|
||||
#include <IO/readFloatText.h>
|
||||
|
||||
#include <Core/Block.h>
|
||||
#include <Common/assert_cast.h>
|
||||
@ -865,6 +866,13 @@ namespace
|
||||
return std::make_shared<DataTypeTuple>(nested_types);
|
||||
}
|
||||
|
||||
bool tryReadFloat(Float64 & value, ReadBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
if (settings.try_infer_exponent_floats)
|
||||
return tryReadFloatText(value, buf);
|
||||
return tryReadFloatTextNoExponent(value, buf);
|
||||
}
|
||||
|
||||
DataTypePtr tryInferNumber(ReadBuffer & buf, const FormatSettings & settings)
|
||||
{
|
||||
if (buf.eof())
|
||||
@ -903,7 +911,7 @@ namespace
|
||||
buf.position() = number_start;
|
||||
}
|
||||
|
||||
if (tryReadFloatText(tmp_float, buf))
|
||||
if (tryReadFloat(tmp_float, buf, settings))
|
||||
{
|
||||
if (read_int && buf.position() == int_end)
|
||||
return std::make_shared<DataTypeInt64>();
|
||||
@ -937,7 +945,7 @@ namespace
|
||||
peekable_buf.rollbackToCheckpoint(true);
|
||||
}
|
||||
|
||||
if (tryReadFloatText(tmp_float, peekable_buf))
|
||||
if (tryReadFloat(tmp_float, peekable_buf, settings))
|
||||
{
|
||||
/// Float parsing reads no fewer bytes than integer parsing,
|
||||
/// so position of the buffer is either the same, or further.
|
||||
@ -949,7 +957,7 @@ namespace
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
}
|
||||
else if (tryReadFloatText(tmp_float, buf))
|
||||
else if (tryReadFloat(tmp_float, buf, settings))
|
||||
{
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
@ -1390,7 +1398,7 @@ DataTypePtr tryInferNumberFromString(std::string_view field, const FormatSetting
|
||||
buf.position() = buf.buffer().begin();
|
||||
|
||||
Float64 tmp;
|
||||
if (tryReadFloatText(tmp, buf) && buf.eof())
|
||||
if (tryReadFloat(tmp, buf, settings) && buf.eof())
|
||||
return std::make_shared<DataTypeFloat64>();
|
||||
|
||||
return nullptr;
|
||||
|
@ -35,7 +35,7 @@ struct EmptyImpl
|
||||
/// Only make sense if is_fixed_to_constant.
|
||||
static void vectorFixedToConstant(const ColumnString::Chars & /*data*/, size_t /*n*/, UInt8 & /*res*/)
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: 'vectorFixedToConstant method' is called");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "'vectorFixedToConstant method' is called");
|
||||
}
|
||||
|
||||
static void vectorFixedToVector(const ColumnString::Chars & data, size_t n, PaddedPODArray<UInt8> & res)
|
||||
|
@ -147,11 +147,32 @@ private: /// it's not correct for Decimal
|
||||
public:
|
||||
static constexpr bool allow_decimal = IsOperation<Operation>::allow_decimal;
|
||||
|
||||
using DecimalResultDataType = Switch<
|
||||
Case<!allow_decimal, InvalidType>,
|
||||
Case<IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType> && UseLeftDecimal<LeftDataType, RightDataType>, LeftDataType>,
|
||||
Case<IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>, RightDataType>,
|
||||
Case<IsDataTypeDecimal<LeftDataType> && IsIntegralOrExtended<RightDataType>, LeftDataType>,
|
||||
Case<IsDataTypeDecimal<RightDataType> && IsIntegralOrExtended<LeftDataType>, RightDataType>,
|
||||
|
||||
/// e.g Decimal +-*/ Float, least(Decimal, Float), greatest(Decimal, Float) = Float64
|
||||
Case<IsDataTypeDecimal<LeftDataType> && IsFloatingPoint<RightDataType>, DataTypeFloat64>,
|
||||
Case<IsDataTypeDecimal<RightDataType> && IsFloatingPoint<LeftDataType>, DataTypeFloat64>,
|
||||
|
||||
Case<IsOperation<Operation>::bit_hamming_distance && IsIntegral<LeftDataType> && IsIntegral<RightDataType>, DataTypeUInt8>,
|
||||
Case<IsOperation<Operation>::bit_hamming_distance && IsFixedString<LeftDataType> && IsFixedString<RightDataType>, DataTypeUInt16>,
|
||||
Case<IsOperation<Operation>::bit_hamming_distance && IsString<LeftDataType> && IsString<RightDataType>, DataTypeUInt64>,
|
||||
|
||||
/// Decimal <op> Real is not supported (traditional DBs convert Decimal <op> Real to Real)
|
||||
Case<IsDataTypeDecimal<LeftDataType> && !IsIntegralOrExtendedOrDecimal<RightDataType>, InvalidType>,
|
||||
Case<IsDataTypeDecimal<RightDataType> && !IsIntegralOrExtendedOrDecimal<LeftDataType>, InvalidType>>;
|
||||
|
||||
/// Appropriate result type for binary operator on numeric types. "Date" can also mean
|
||||
/// DateTime, but if both operands are Dates, their type must be the same (e.g. Date - DateTime is invalid).
|
||||
using ResultDataType = Switch<
|
||||
/// Result must be Integer
|
||||
Case<IsOperation<Operation>::div_int || IsOperation<Operation>::div_int_or_zero, DataTypeFromFieldType<typename Op::ResultType>>,
|
||||
/// Decimal cases
|
||||
Case<!allow_decimal && (IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>), InvalidType>,
|
||||
Case<IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>, DecimalResultDataType>,
|
||||
Case<
|
||||
IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType> && UseLeftDecimal<LeftDataType, RightDataType>,
|
||||
LeftDataType>,
|
||||
@ -622,7 +643,11 @@ private:
|
||||
if constexpr (op_case == OpCase::RightConstant)
|
||||
{
|
||||
if ((*right_nullmap)[0])
|
||||
{
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = ResultType();
|
||||
return;
|
||||
}
|
||||
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
c[i] = apply_func(undec(a[i]), undec(b));
|
||||
@ -1665,7 +1690,9 @@ public:
|
||||
|
||||
if constexpr (!std::is_same_v<ResultDataType, InvalidType>)
|
||||
{
|
||||
if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
|
||||
if constexpr (is_div_int || is_div_int_or_zero)
|
||||
type_res = std::make_shared<ResultDataType>();
|
||||
else if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
|
||||
{
|
||||
if constexpr (is_division)
|
||||
{
|
||||
@ -1685,13 +1712,19 @@ public:
|
||||
ResultDataType result_type = decimalResultType<is_multiply, is_division>(left, right);
|
||||
type_res = std::make_shared<ResultDataType>(result_type.getPrecision(), result_type.getScale());
|
||||
}
|
||||
else if constexpr ((IsDataTypeDecimal<LeftDataType> && IsFloatingPoint<RightDataType>) ||
|
||||
(IsDataTypeDecimal<RightDataType> && IsFloatingPoint<LeftDataType>))
|
||||
else if constexpr (((IsDataTypeDecimal<LeftDataType> && IsFloatingPoint<RightDataType>) ||
|
||||
(IsDataTypeDecimal<RightDataType> && IsFloatingPoint<LeftDataType>)))
|
||||
{
|
||||
type_res = std::make_shared<DataTypeFloat64>();
|
||||
}
|
||||
else if constexpr (IsDataTypeDecimal<LeftDataType>)
|
||||
{
|
||||
type_res = std::make_shared<LeftDataType>(left.getPrecision(), left.getScale());
|
||||
}
|
||||
else if constexpr (IsDataTypeDecimal<RightDataType>)
|
||||
{
|
||||
type_res = std::make_shared<RightDataType>(right.getPrecision(), right.getScale());
|
||||
}
|
||||
else if constexpr (std::is_same_v<ResultDataType, DataTypeDateTime>)
|
||||
{
|
||||
// Special case for DateTime: binary OPS should reuse timezone
|
||||
@ -2000,6 +2033,7 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
|
||||
using LeftDataType = std::decay_t<decltype(left)>;
|
||||
using RightDataType = std::decay_t<decltype(right)>;
|
||||
using ResultDataType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::ResultDataType;
|
||||
using DecimalResultType = typename BinaryOperationTraits<Op, LeftDataType, RightDataType>::DecimalResultDataType;
|
||||
|
||||
if constexpr (std::is_same_v<ResultDataType, InvalidType>)
|
||||
return nullptr;
|
||||
@ -2051,6 +2085,35 @@ ColumnPtr executeStringInteger(const ColumnsWithTypeAndName & arguments, const A
|
||||
col_left_size,
|
||||
right_nullmap);
|
||||
}
|
||||
/// Here we check if we have `intDiv` or `intDivOrZero` and at least one of the arguments is decimal, because in this case originally we had result as decimal, so we need to convert result into integer after calculations
|
||||
else if constexpr (!decimal_with_float && (is_div_int || is_div_int_or_zero) && (IsDataTypeDecimal<LeftDataType> || IsDataTypeDecimal<RightDataType>))
|
||||
{
|
||||
|
||||
if constexpr (!std::is_same_v<DecimalResultType, InvalidType>)
|
||||
{
|
||||
DataTypePtr type_res;
|
||||
if constexpr (IsDataTypeDecimal<LeftDataType> && IsDataTypeDecimal<RightDataType>)
|
||||
{
|
||||
DecimalResultType result_type = decimalResultType<is_multiply, is_division>(left, right);
|
||||
type_res = std::make_shared<DecimalResultType>(result_type.getPrecision(), result_type.getScale());
|
||||
}
|
||||
else if constexpr (IsDataTypeDecimal<LeftDataType>)
|
||||
type_res = std::make_shared<LeftDataType>(left.getPrecision(), left.getScale());
|
||||
else
|
||||
type_res = std::make_shared<RightDataType>(right.getPrecision(), right.getScale());
|
||||
|
||||
auto res = executeNumericWithDecimal<LeftDataType, RightDataType, DecimalResultType>(
|
||||
left, right,
|
||||
col_left_const, col_right_const,
|
||||
col_left, col_right,
|
||||
col_left_size,
|
||||
right_nullmap);
|
||||
|
||||
auto col = ColumnWithTypeAndName(res, type_res, name);
|
||||
return castColumn(col, std::make_shared<ResultDataType>());
|
||||
}
|
||||
return nullptr;
|
||||
}
|
||||
else // can't avoid else and another indentation level, otherwise the compiler would try to instantiate
|
||||
// ColVecResult for Decimals which would lead to a compile error.
|
||||
{
|
||||
|
@ -811,7 +811,7 @@ private:
|
||||
c0_const_size = c0_const_fixed_string->getN();
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column");
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column");
|
||||
}
|
||||
|
||||
if (c1_const)
|
||||
@ -830,7 +830,7 @@ private:
|
||||
c1_const_size = c1_const_fixed_string->getN();
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Logical error: ColumnConst contains not String nor FixedString column");
|
||||
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ColumnConst contains not String nor FixedString column");
|
||||
}
|
||||
|
||||
using StringImpl = StringComparisonImpl<Op<int, int>>;
|
||||
@ -1114,7 +1114,7 @@ private:
|
||||
/// This is a paranoid check to protect from a broken query analysis.
|
||||
if (c0->isNullable() != c1->isNullable())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Logical error: columns are assumed to be of identical types, but they are different in Nullable");
|
||||
"Columns are assumed to be of identical types, but they are different in Nullable");
|
||||
|
||||
if (c0_const && c1_const)
|
||||
{
|
||||
|
@ -2560,7 +2560,7 @@ public:
|
||||
if constexpr (std::is_same_v<ToDataType, DataTypeDateTime>)
|
||||
res = std::make_shared<DataTypeDateTime>(extractTimeZoneNameFromFunctionArguments(arguments, 1, 0, false));
|
||||
else if constexpr (std::is_same_v<ToDataType, DataTypeDateTime64>)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: It is a bug.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MaterializedMySQL is a bug.");
|
||||
else if constexpr (to_decimal)
|
||||
{
|
||||
UInt64 scale = extractToDecimalScale(arguments[1]);
|
||||
|
@ -5,7 +5,7 @@
|
||||
|
||||
#include <base/range.h>
|
||||
|
||||
#include <Common/CpuId.h>
|
||||
#include <Common/CPUID.h>
|
||||
#include <Common/typeid_cast.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
|
@ -530,7 +530,7 @@ DataTypePtr FunctionAnyArityLogical<Impl, Name>::getReturnTypeImpl(const DataTyp
|
||||
{
|
||||
has_nullable_arguments = arg_type->isNullable();
|
||||
if (has_nullable_arguments && !Impl::specialImplementationForNulls())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: Unexpected type of argument for function \"{}\": "
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected type of argument for function \"{}\": "
|
||||
" argument {} is of type {}", getName(), i + 1, arg_type->getName());
|
||||
}
|
||||
|
||||
|
@ -61,7 +61,7 @@ struct IsOperation
|
||||
static constexpr bool bit_hamming_distance = IsSameOperation<Op, BitHammingDistanceImpl>::value;
|
||||
|
||||
static constexpr bool division = div_floating || div_int || div_int_or_zero || modulo;
|
||||
|
||||
// NOTE: allow_decimal should not fully contain `division` because of divInt
|
||||
static constexpr bool allow_decimal = plus || minus || multiply || division || least || greatest;
|
||||
};
|
||||
|
||||
|
@ -17,7 +17,7 @@ namespace DB
|
||||
namespace
|
||||
{
|
||||
|
||||
class FunctionAddressToLine: public FunctionAddressToLineBase<StringRef, Dwarf::LocationInfoMode::FAST>
|
||||
class FunctionAddressToLine : public FunctionAddressToLineBase<StringRef, Dwarf::LocationInfoMode::FAST>
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "addressToLine";
|
||||
|
@ -1,5 +1,5 @@
|
||||
#include "divide.h"
|
||||
#include <Common/CpuId.h>
|
||||
#include <Common/CPUID.h>
|
||||
|
||||
#if defined(__x86_64__)
|
||||
namespace SSE2
|
||||
@ -26,9 +26,9 @@ template <typename A, typename B, typename ResultType>
|
||||
void divideImpl(const A * __restrict a_pos, B b, ResultType * __restrict c_pos, size_t size)
|
||||
{
|
||||
#if defined(__x86_64__)
|
||||
if (DB::Cpu::CpuFlagsCache::have_AVX2)
|
||||
if (DB::CPU::CPUFlagsCache::have_AVX2)
|
||||
AVX2::divideImpl(a_pos, b, c_pos, size);
|
||||
else if (DB::Cpu::CpuFlagsCache::have_SSE2)
|
||||
else if (DB::CPU::CPUFlagsCache::have_SSE2)
|
||||
SSE2::divideImpl(a_pos, b, c_pos, size);
|
||||
#else
|
||||
Generic::divideImpl(a_pos, b, c_pos, size);
|
||||
|
@ -51,12 +51,12 @@ namespace
|
||||
};
|
||||
|
||||
|
||||
class FunctionTcpPort : public FunctionConstantBase<FunctionTcpPort, UInt16, DataTypeUInt16>
|
||||
class FunctionTCPPort : public FunctionConstantBase<FunctionTCPPort, UInt16, DataTypeUInt16>
|
||||
{
|
||||
public:
|
||||
static constexpr auto name = "tcpPort";
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTcpPort>(context); }
|
||||
explicit FunctionTcpPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {}
|
||||
static FunctionPtr create(ContextPtr context) { return std::make_shared<FunctionTCPPort>(context); }
|
||||
explicit FunctionTCPPort(ContextPtr context) : FunctionConstantBase(context->getTCPPort(), context->isDistributed()) {}
|
||||
};
|
||||
|
||||
|
||||
@ -153,9 +153,9 @@ REGISTER_FUNCTION(ServerUUID)
|
||||
factory.registerFunction<FunctionServerUUID>();
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(TcpPort)
|
||||
REGISTER_FUNCTION(TCPPort)
|
||||
{
|
||||
factory.registerFunction<FunctionTcpPort>();
|
||||
factory.registerFunction<FunctionTCPPort>();
|
||||
}
|
||||
|
||||
REGISTER_FUNCTION(Timezone)
|
||||
|
@ -177,7 +177,7 @@ public:
|
||||
}
|
||||
else if (mode == "logical error")
|
||||
{
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: trap");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Trap");
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Unknown trap mode");
|
||||
|
@ -4,8 +4,8 @@
|
||||
|
||||
#if USE_AZURE_BLOB_STORAGE
|
||||
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Disks/ObjectStorages/AzureBlobStorage/AzureObjectStorage.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <base/types.h>
|
||||
#include <functional>
|
||||
#include <memory>
|
||||
|
@ -20,7 +20,7 @@ ConnectionTimeouts ConnectionTimeouts::getTCPTimeoutsWithoutFailover(const Setti
|
||||
.withConnectionTimeout(settings.connect_timeout)
|
||||
.withSendTimeout(settings.send_timeout)
|
||||
.withReceiveTimeout(settings.receive_timeout)
|
||||
.withTcpKeepAliveTimeout(settings.tcp_keep_alive_timeout)
|
||||
.withTCPKeepAliveTimeout(settings.tcp_keep_alive_timeout)
|
||||
.withHandshakeTimeout(settings.handshake_timeout_ms)
|
||||
.withHedgedConnectionTimeout(settings.hedged_connection_timeout_ms)
|
||||
.withReceiveDataTimeout(settings.receive_data_timeout_ms);
|
||||
@ -40,8 +40,8 @@ ConnectionTimeouts ConnectionTimeouts::getHTTPTimeouts(const Settings & settings
|
||||
.withConnectionTimeout(settings.http_connection_timeout)
|
||||
.withSendTimeout(settings.http_send_timeout)
|
||||
.withReceiveTimeout(settings.http_receive_timeout)
|
||||
.withHttpKeepAliveTimeout(http_keep_alive_timeout)
|
||||
.withTcpKeepAliveTimeout(settings.tcp_keep_alive_timeout)
|
||||
.withHTTPKeepAliveTimeout(http_keep_alive_timeout)
|
||||
.withTCPKeepAliveTimeout(settings.tcp_keep_alive_timeout)
|
||||
.withHandshakeTimeout(settings.handshake_timeout_ms);
|
||||
}
|
||||
|
||||
|
@ -16,8 +16,8 @@ struct Settings;
|
||||
M(secure_connection_timeout, withSecureConnectionTimeout) \
|
||||
M(send_timeout, withSendTimeout) \
|
||||
M(receive_timeout, withReceiveTimeout) \
|
||||
M(tcp_keep_alive_timeout, withTcpKeepAliveTimeout) \
|
||||
M(http_keep_alive_timeout, withHttpKeepAliveTimeout) \
|
||||
M(tcp_keep_alive_timeout, withTCPKeepAliveTimeout) \
|
||||
M(http_keep_alive_timeout, withHTTPKeepAliveTimeout) \
|
||||
M(hedged_connection_timeout, withHedgedConnectionTimeout) \
|
||||
M(receive_data_timeout, withReceiveDataTimeout) \
|
||||
M(handshake_timeout, withHandshakeTimeout) \
|
||||
|
@ -3,7 +3,7 @@
|
||||
#include <IO/BufferWithOwnMemory.h>
|
||||
#include <IO/ReadBuffer.h>
|
||||
#include <IO/SeekableReadBuffer.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <Common/ArenaWithFreeLists.h>
|
||||
|
||||
namespace DB
|
||||
|
@ -22,7 +22,6 @@ namespace ErrorCodes
|
||||
# include <aws/core/utils/UUID.h>
|
||||
# include <aws/core/http/HttpClientFactory.h>
|
||||
|
||||
# include <IO/S3/PocoHTTPClientFactory.h>
|
||||
# include <aws/core/utils/HashingUtils.h>
|
||||
# include <aws/core/platform/FileSystem.h>
|
||||
|
||||
@ -31,9 +30,7 @@ namespace ErrorCodes
|
||||
# include <IO/S3/Client.h>
|
||||
|
||||
# include <fstream>
|
||||
# include <base/EnumReflection.h>
|
||||
|
||||
# include <boost/algorithm/string.hpp>
|
||||
# include <boost/algorithm/string/split.hpp>
|
||||
# include <boost/algorithm/string/classification.hpp>
|
||||
# include <Poco/Exception.h>
|
||||
@ -755,7 +752,7 @@ S3CredentialsProviderChain::S3CredentialsProviderChain(
|
||||
configuration.put_request_throttler,
|
||||
Aws::Http::SchemeMapper::ToString(Aws::Http::Scheme::HTTP));
|
||||
|
||||
/// See MakeDefaultHttpResourceClientConfiguration().
|
||||
/// See MakeDefaultHTTPResourceClientConfiguration().
|
||||
/// This is part of EC2 metadata client, but unfortunately it can't be accessed from outside
|
||||
/// of contrib/aws/aws-cpp-sdk-core/source/internal/AWSHttpResourceClient.cpp
|
||||
aws_client_configuration.maxConnections = 2;
|
||||
|
@ -146,9 +146,9 @@ ConnectionTimeouts getTimeoutsFromConfiguration(const PocoHTTPClientConfiguratio
|
||||
.withConnectionTimeout(Poco::Timespan(client_configuration.connectTimeoutMs * 1000))
|
||||
.withSendTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
|
||||
.withReceiveTimeout(Poco::Timespan(client_configuration.requestTimeoutMs * 1000))
|
||||
.withTcpKeepAliveTimeout(Poco::Timespan(
|
||||
.withTCPKeepAliveTimeout(Poco::Timespan(
|
||||
client_configuration.enableTcpKeepAlive ? client_configuration.tcpKeepAliveIntervalMs * 1000 : 0))
|
||||
.withHttpKeepAliveTimeout(Poco::Timespan(
|
||||
.withHTTPKeepAliveTimeout(Poco::Timespan(
|
||||
client_configuration.http_keep_alive_timeout_ms * 1000)); /// flag indicating whether keep-alive is enabled is set to each session upon creation
|
||||
}
|
||||
|
||||
|
@ -5,7 +5,7 @@
|
||||
#if USE_AWS_S3
|
||||
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
#include <base/types.h>
|
||||
#include <functional>
|
||||
|
@ -10,7 +10,7 @@
|
||||
#include <IO/WriteBuffer.h>
|
||||
#include <IO/WriteSettings.h>
|
||||
#include <Storages/StorageS3Settings.h>
|
||||
#include <Interpreters/threadPoolCallbackRunner.h>
|
||||
#include <Common/threadPoolCallbackRunner.h>
|
||||
#include <IO/S3/BlobStorageLogWriter.h>
|
||||
|
||||
#include <memory>
|
||||
|
@ -67,4 +67,7 @@ template void readFloatText<Float64>(Float64 &, ReadBuffer &);
|
||||
template bool tryReadFloatText<Float32>(Float32 &, ReadBuffer &);
|
||||
template bool tryReadFloatText<Float64>(Float64 &, ReadBuffer &);
|
||||
|
||||
template bool tryReadFloatTextNoExponent<Float32>(Float32 &, ReadBuffer &);
|
||||
template bool tryReadFloatTextNoExponent<Float64>(Float64 &, ReadBuffer &);
|
||||
|
||||
}
|
||||
|
@ -324,7 +324,7 @@ static inline void readUIntTextUpToNSignificantDigits(T & x, ReadBuffer & buf)
|
||||
}
|
||||
|
||||
|
||||
template <typename T, typename ReturnType>
|
||||
template <typename T, typename ReturnType, bool allow_exponent = true>
|
||||
ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
|
||||
{
|
||||
static_assert(std::is_same_v<T, double> || std::is_same_v<T, float>, "Argument for readFloatTextImpl must be float or double");
|
||||
@ -395,30 +395,33 @@ ReturnType readFloatTextFastImpl(T & x, ReadBuffer & in)
|
||||
after_point_exponent = (read_digits > significant_digits ? -significant_digits : static_cast<int>(-read_digits)) - after_point_num_leading_zeros;
|
||||
}
|
||||
|
||||
if (checkChar('e', in) || checkChar('E', in))
|
||||
if constexpr (allow_exponent)
|
||||
{
|
||||
if (in.eof())
|
||||
if (checkChar('e', in) || checkChar('E', in))
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent");
|
||||
else
|
||||
return false;
|
||||
}
|
||||
if (in.eof())
|
||||
{
|
||||
if constexpr (throw_exception)
|
||||
throw Exception(ErrorCodes::CANNOT_PARSE_NUMBER, "Cannot read floating point value: nothing after exponent");
|
||||
else
|
||||
return false;
|
||||
}
|
||||
|
||||
bool exponent_negative = false;
|
||||
if (*in.position() == '-')
|
||||
{
|
||||
exponent_negative = true;
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '+')
|
||||
{
|
||||
++in.position();
|
||||
}
|
||||
bool exponent_negative = false;
|
||||
if (*in.position() == '-')
|
||||
{
|
||||
exponent_negative = true;
|
||||
++in.position();
|
||||
}
|
||||
else if (*in.position() == '+')
|
||||
{
|
||||
++in.position();
|
||||
}
|
||||
|
||||
readUIntTextUpToNSignificantDigits<4>(exponent, in);
|
||||
if (exponent_negative)
|
||||
exponent = -exponent;
|
||||
readUIntTextUpToNSignificantDigits<4>(exponent, in);
|
||||
if (exponent_negative)
|
||||
exponent = -exponent;
|
||||
}
|
||||
}
|
||||
|
||||
if (after_point)
|
||||
@ -604,4 +607,7 @@ template <typename T> bool tryReadFloatTextSimple(T & x, ReadBuffer & in) { retu
|
||||
template <typename T> void readFloatText(T & x, ReadBuffer & in) { readFloatTextFast(x, in); }
|
||||
template <typename T> bool tryReadFloatText(T & x, ReadBuffer & in) { return tryReadFloatTextFast(x, in); }
|
||||
|
||||
/// Don't read exponent part of the number.
|
||||
template <typename T> bool tryReadFloatTextNoExponent(T & x, ReadBuffer & in) { return readFloatTextFastImpl<T, bool, false>(x, in); }
|
||||
|
||||
}
|
||||
|
@ -624,7 +624,7 @@ Aggregator::Aggregator(const Block & header_, const Params & params_)
|
||||
{
|
||||
size_t alignment_of_next_state = params.aggregates[i + 1].function->alignOfData();
|
||||
if ((alignment_of_next_state & (alignment_of_next_state - 1)) != 0)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: alignOfData is not 2^N");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "`alignOfData` is not 2^N");
|
||||
|
||||
/// Extend total_size to next alignment requirement
|
||||
/// Add padding by rounding up 'total_size_of_aggregate_states' to be a multiplier of alignment_of_next_state.
|
||||
@ -857,7 +857,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
||||
return AggregatedDataVariants::Type::low_cardinality_keys128;
|
||||
if (size_of_field == 32)
|
||||
return AggregatedDataVariants::Type::low_cardinality_keys256;
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: low cardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LowCardinality numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
}
|
||||
|
||||
if (size_of_field == 1)
|
||||
@ -872,7 +872,7 @@ AggregatedDataVariants::Type Aggregator::chooseAggregationMethod()
|
||||
return AggregatedDataVariants::Type::keys128;
|
||||
if (size_of_field == 32)
|
||||
return AggregatedDataVariants::Type::keys256;
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
}
|
||||
|
||||
if (params.keys_size == 1 && isFixedString(types_removed_nullable[0]))
|
||||
|
@ -62,7 +62,7 @@ private:
|
||||
{
|
||||
auto [array_join_expression_list, _] = node.arrayJoinExpressionList();
|
||||
if (!array_join_expression_list)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no ARRAY JOIN");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No ARRAY JOIN");
|
||||
|
||||
std::vector<ASTPtr *> out;
|
||||
out.reserve(array_join_expression_list->children.size());
|
||||
|
@ -23,7 +23,7 @@ namespace ErrorCodes
|
||||
void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
|
||||
{
|
||||
if (server_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::write is called for unsupported server revision");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::write is called for unsupported server revision");
|
||||
|
||||
writeBinary(static_cast<UInt8>(query_kind), out);
|
||||
if (empty())
|
||||
@ -103,7 +103,7 @@ void ClientInfo::write(WriteBuffer & out, UInt64 server_protocol_revision) const
|
||||
void ClientInfo::read(ReadBuffer & in, UInt64 client_protocol_revision)
|
||||
{
|
||||
if (client_protocol_revision < DBMS_MIN_REVISION_WITH_CLIENT_INFO)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: method ClientInfo::read is called for unsupported client revision");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Method ClientInfo::read is called for unsupported client revision");
|
||||
|
||||
UInt8 read_query_kind = 0;
|
||||
readBinary(read_query_kind, in);
|
||||
|
@ -94,6 +94,7 @@
|
||||
#include <Common/logger_useful.h>
|
||||
#include <Common/RemoteHostFilter.h>
|
||||
#include <Common/HTTPHeaderFilter.h>
|
||||
#include <Interpreters/InterpreterSelectQueryAnalyzer.h>
|
||||
#include <Interpreters/AsynchronousInsertQueue.h>
|
||||
#include <Interpreters/DatabaseCatalog.h>
|
||||
#include <Interpreters/JIT/CompiledExpressionCache.h>
|
||||
@ -1533,7 +1534,7 @@ void Context::addExternalTable(const String & table_name, TemporaryTableHolder &
|
||||
|
||||
std::lock_guard lock(mutex);
|
||||
if (external_tables_mapping.end() != external_tables_mapping.find(table_name))
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} already exists.", backQuoteIfNeed(table_name));
|
||||
throw Exception(ErrorCodes::TABLE_ALREADY_EXISTS, "Temporary table {} already exists", backQuoteIfNeed(table_name));
|
||||
external_tables_mapping.emplace(table_name, std::make_shared<TemporaryTableHolder>(std::move(temporary_table)));
|
||||
}
|
||||
|
||||
@ -1931,6 +1932,35 @@ StoragePtr Context::executeTableFunction(const ASTPtr & table_expression, const
|
||||
}
|
||||
|
||||
|
||||
StoragePtr Context::buildParametrizedViewStorage(const ASTPtr & table_expression, const String & database_name, const String & table_name)
|
||||
{
|
||||
if (table_name.empty())
|
||||
return nullptr;
|
||||
|
||||
StoragePtr original_view = DatabaseCatalog::instance().tryGetTable({database_name, table_name}, getQueryContext());
|
||||
if (!original_view || !original_view->isView())
|
||||
return nullptr;
|
||||
auto * storage_view = original_view->as<StorageView>();
|
||||
if (!storage_view || !storage_view->isParameterizedView())
|
||||
return nullptr;
|
||||
|
||||
auto query = original_view->getInMemoryMetadataPtr()->getSelectQuery().inner_query->clone();
|
||||
NameToNameMap parameterized_view_values = analyzeFunctionParamValues(table_expression);
|
||||
StorageView::replaceQueryParametersIfParametrizedView(query, parameterized_view_values);
|
||||
|
||||
ASTCreateQuery create;
|
||||
create.select = query->as<ASTSelectWithUnionQuery>();
|
||||
auto sample_block = InterpreterSelectQueryAnalyzer::getSampleBlock(query, shared_from_this());
|
||||
auto res = std::make_shared<StorageView>(StorageID(database_name, table_name),
|
||||
create,
|
||||
ColumnsDescription(sample_block.getNamesAndTypesList()),
|
||||
/* comment */ "",
|
||||
/* is_parameterized_view */ true);
|
||||
res->startup();
|
||||
return res;
|
||||
}
|
||||
|
||||
|
||||
void Context::addViewSource(const StoragePtr & storage)
|
||||
{
|
||||
if (view_source)
|
||||
@ -4483,7 +4513,7 @@ void Context::setClientConnectionId(uint32_t connection_id_)
|
||||
client_info.connection_id = connection_id_;
|
||||
}
|
||||
|
||||
void Context::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
void Context::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
{
|
||||
client_info.http_method = http_method;
|
||||
client_info.http_user_agent = http_user_agent;
|
||||
|
@ -630,7 +630,7 @@ public:
|
||||
void setClientInterface(ClientInfo::Interface interface);
|
||||
void setClientVersion(UInt64 client_version_major, UInt64 client_version_minor, UInt64 client_version_patch, unsigned client_tcp_protocol_version);
|
||||
void setClientConnectionId(uint32_t connection_id);
|
||||
void setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer);
|
||||
void setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer);
|
||||
void setForwardedFor(const String & forwarded_for);
|
||||
void setQueryKind(ClientInfo::QueryKind query_kind);
|
||||
void setQueryKindInitial();
|
||||
@ -718,6 +718,8 @@ public:
|
||||
/// Overload for the new analyzer. Structure inference is performed in QueryAnalysisPass.
|
||||
StoragePtr executeTableFunction(const ASTPtr & table_expression, const TableFunctionPtr & table_function_ptr);
|
||||
|
||||
StoragePtr buildParametrizedViewStorage(const ASTPtr & table_expression, const String & database_name, const String & table_name);
|
||||
|
||||
void addViewSource(const StoragePtr & storage);
|
||||
StoragePtr getViewSource() const;
|
||||
|
||||
|
@ -173,7 +173,7 @@ std::vector<JoinedElement> getTables(const ASTSelectQuery & select)
|
||||
{
|
||||
const auto * table_element = child->as<ASTTablesInSelectQueryElement>();
|
||||
if (!table_element)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: TablesInSelectQueryElement expected");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "TablesInSelectQueryElement expected");
|
||||
|
||||
JoinedElement & t = joined_tables.emplace_back(*table_element);
|
||||
t.rewriteCommaToCross();
|
||||
@ -224,7 +224,7 @@ void CrossToInnerJoinMatcher::visit(ASTSelectQuery & select, ASTPtr &, Data & da
|
||||
{
|
||||
if (joined_tables.size() != data.tables_with_columns.size())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR,
|
||||
"Logical error: inconsistent number of tables: {} != {}",
|
||||
"Inconsistent number of tables: {} != {}",
|
||||
joined_tables.size(), data.tables_with_columns.size());
|
||||
|
||||
for (size_t i = 0; i < joined_tables.size(); ++i)
|
||||
|
@ -148,9 +148,8 @@ void DDLLogEntry::parse(const String & data)
|
||||
String settings_str;
|
||||
rb >> "settings: " >> settings_str >> "\n";
|
||||
ParserSetQuery parser{true};
|
||||
constexpr UInt64 max_size = 4096;
|
||||
constexpr UInt64 max_depth = 16;
|
||||
ASTPtr settings_ast = parseQuery(parser, settings_str, max_size, max_depth);
|
||||
ASTPtr settings_ast = parseQuery(parser, settings_str, Context::getGlobalContextInstance()->getSettingsRef().max_query_size, max_depth);
|
||||
settings.emplace(std::move(settings_ast->as<ASTSetQuery>()->changes));
|
||||
}
|
||||
}
|
||||
|
@ -71,7 +71,7 @@ DatabaseAndTableWithAlias::DatabaseAndTableWithAlias(const ASTTableExpression &
|
||||
alias = table_expression.subquery->tryGetAlias();
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no known elements in ASTTableExpression");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No known elements in ASTTableExpression");
|
||||
}
|
||||
|
||||
bool DatabaseAndTableWithAlias::satisfies(const DatabaseAndTableWithAlias & db_table, bool table_may_be_an_alias) const
|
||||
|
@ -368,7 +368,7 @@ HashJoin::Type HashJoin::chooseMethod(JoinKind kind, const ColumnRawPtrs & key_c
|
||||
return Type::keys128;
|
||||
if (size_of_field == 32)
|
||||
return Type::keys256;
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Numeric column has sizeOfField not in 1, 2, 4, 8, 16, 32.");
|
||||
}
|
||||
|
||||
/// If the keys fit in N bits, we will use a hash table for N-bit-packed keys
|
||||
|
@ -103,12 +103,12 @@ private:
|
||||
/// Already processed.
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected function name {}", concrete->name);
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected function name {}", concrete->name);
|
||||
}
|
||||
else if (table_join)
|
||||
table_join->locality = JoinLocality::Global;
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: unexpected AST node");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected AST node");
|
||||
}
|
||||
else if (distributed_product_mode == DistributedProductMode::DENY)
|
||||
{
|
||||
|
@ -82,7 +82,8 @@
|
||||
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionFactory.h>
|
||||
#include <Functions/UserDefined/UserDefinedSQLFunctionVisitor.h>
|
||||
|
||||
#include <Interpreters/ReplaceQueryParameterVisitor.h>
|
||||
#include <Parsers/QueryParameterVisitor.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -692,6 +693,7 @@ ColumnsDescription InterpreterCreateQuery::getColumnsDescription(
|
||||
if (!attach && !is_restore_from_backup && context_->getSettingsRef().flatten_nested)
|
||||
res.flattenNested();
|
||||
|
||||
|
||||
if (res.getAllPhysical().empty())
|
||||
throw Exception(ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED, "Cannot CREATE table without physical columns");
|
||||
|
||||
@ -796,6 +798,9 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
}
|
||||
else if (create.select)
|
||||
{
|
||||
if (create.isParameterizedView())
|
||||
return properties;
|
||||
|
||||
Block as_select_sample;
|
||||
|
||||
if (getContext()->getSettingsRef().allow_experimental_analyzer)
|
||||
@ -820,11 +825,7 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
|
||||
* for example: LIMIT, OFFSET, functions parameters, functions constant only arguments.
|
||||
*/
|
||||
|
||||
SelectQueryOptions options;
|
||||
if (create.isParameterizedView())
|
||||
options = options.createParameterizedView();
|
||||
|
||||
InterpreterSelectWithUnionQuery interpreter(create.select->clone(), getContext(), options);
|
||||
InterpreterSelectWithUnionQuery interpreter(create.select->clone(), getContext(), SelectQueryOptions());
|
||||
as_select_sample = interpreter.getSampleBlock();
|
||||
}
|
||||
|
||||
|
@ -56,7 +56,7 @@ InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery(
|
||||
|
||||
size_t num_children = ast->list_of_selects->children.size();
|
||||
if (!num_children)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: no children in ASTSelectWithUnionQuery");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "No children in ASTSelectWithUnionQuery");
|
||||
|
||||
/// Note that we pass 'required_result_column_names' to first SELECT.
|
||||
/// And for the rest, we pass names at the corresponding positions of 'required_result_column_names' in the result of first SELECT,
|
||||
|
@ -25,13 +25,13 @@ String InterpreterShowFunctionsQuery::getRewrittenQuery()
|
||||
|
||||
const auto & query = query_ptr->as<ASTShowFunctionsQuery &>();
|
||||
|
||||
DatabasePtr systemDb = DatabaseCatalog::instance().getSystemDatabase();
|
||||
DatabasePtr system_db = DatabaseCatalog::instance().getSystemDatabase();
|
||||
|
||||
String rewritten_query = fmt::format(
|
||||
R"(
|
||||
SELECT *
|
||||
FROM {}.{})",
|
||||
systemDb->getDatabaseName(),
|
||||
system_db->getDatabaseName(),
|
||||
functions_table);
|
||||
|
||||
if (!query.like.empty())
|
||||
|
@ -168,7 +168,7 @@ private:
|
||||
has_asterisks = true;
|
||||
|
||||
if (!qualified_asterisk->qualifier)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must have a qualifier");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must have a qualifier");
|
||||
|
||||
auto & identifier = qualified_asterisk->qualifier->as<ASTIdentifier &>();
|
||||
|
||||
@ -183,7 +183,7 @@ private:
|
||||
transformer->as<ASTColumnsReplaceTransformer>())
|
||||
IASTColumnsTransformer::transform(transformer, columns);
|
||||
else
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: qualified asterisk must only have children of IASTColumnsTransformer type");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Qualified asterisk must only have children of IASTColumnsTransformer type");
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -239,7 +239,7 @@ public:
|
||||
|
||||
/// SortCursorImpl can work with permutation, but MergeJoinCursor can't.
|
||||
if (impl.permutation)
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Logical error: MergeJoinCursor doesn't support permutation");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MergeJoinCursor doesn't support permutation");
|
||||
}
|
||||
|
||||
size_t position() const { return impl.getRow(); }
|
||||
|
@ -337,7 +337,7 @@ static ASTPtr getPartitionPolicy(const NamesAndTypesList & primary_keys)
|
||||
WhichDataType which(type);
|
||||
|
||||
if (which.isNullable())
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "LOGICAL ERROR: MySQL primary key must be not null, it is a bug.");
|
||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "MySQL's primary key must be not null, it is a bug.");
|
||||
|
||||
if (which.isDate() || which.isDate32() || which.isDateTime() || which.isDateTime64())
|
||||
{
|
||||
|
@ -295,7 +295,7 @@ ProcessListEntry::~ProcessListEntry()
|
||||
auto user_process_list_it = parent.user_to_queries.find(user);
|
||||
if (user_process_list_it == parent.user_to_queries.end())
|
||||
{
|
||||
LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find user in ProcessList");
|
||||
LOG_ERROR(getLogger("ProcessList"), "Cannot find user in ProcessList");
|
||||
std::terminate();
|
||||
}
|
||||
|
||||
@ -323,7 +323,7 @@ ProcessListEntry::~ProcessListEntry()
|
||||
|
||||
if (!found)
|
||||
{
|
||||
LOG_ERROR(getLogger("ProcessList"), "Logical error: cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
|
||||
LOG_ERROR(getLogger("ProcessList"), "Cannot find query by query_id and pointer to ProcessListElement in ProcessListForUser");
|
||||
std::terminate();
|
||||
}
|
||||
|
||||
|
@ -429,11 +429,11 @@ void Session::setClientConnectionId(uint32_t connection_id)
|
||||
prepared_client_info->connection_id = connection_id;
|
||||
}
|
||||
|
||||
void Session::setHttpClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
void Session::setHTTPClientInfo(ClientInfo::HTTPMethod http_method, const String & http_user_agent, const String & http_referer)
|
||||
{
|
||||
if (session_context)
|
||||
{
|
||||
session_context->setHttpClientInfo(http_method, http_user_agent, http_referer);
|
||||
session_context->setHTTPClientInfo(http_method, http_user_agent, http_referer);
|
||||
}
|
||||
else
|
||||
{
|
||||
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue
Block a user