Merge branch 'master' into atomic_metadata5

This commit is contained in:
alesapin 2020-06-23 11:13:05 +03:00
commit 4185fe9975
120 changed files with 747 additions and 192 deletions

View File

@ -85,7 +85,8 @@ static const size_t signal_pipe_buf_size =
+ sizeof(ucontext_t)
+ sizeof(StackTrace)
+ sizeof(UInt32)
+ max_query_id_size + 1; /// query_id + varint encoded length
+ max_query_id_size + 1 /// query_id + varint encoded length
+ sizeof(void*);
using signal_function = void(int, siginfo_t*, void*);
@ -135,6 +136,7 @@ static void signalHandler(int sig, siginfo_t * info, void * context)
DB::writePODBinary(stack_trace, out);
DB::writeBinary(UInt32(getThreadId()), out);
DB::writeStringBinary(query_id, out);
DB::writePODBinary(DB::current_thread, out);
out.next();
@ -218,16 +220,18 @@ public:
StackTrace stack_trace(NoCapture{});
UInt32 thread_num;
std::string query_id;
DB::ThreadStatus * thread_ptr{};
DB::readPODBinary(info, in);
DB::readPODBinary(context, in);
DB::readPODBinary(stack_trace, in);
DB::readBinary(thread_num, in);
DB::readBinary(query_id, in);
DB::readPODBinary(thread_ptr, in);
/// This allows to receive more signals if failure happens inside onFault function.
/// Example: segfault while symbolizing stack trace.
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id); }).detach();
std::thread([=, this] { onFault(sig, info, context, stack_trace, thread_num, query_id, thread_ptr); }).detach();
}
}
}
@ -248,8 +252,19 @@ private:
const ucontext_t & context,
const StackTrace & stack_trace,
UInt32 thread_num,
const std::string & query_id) const
const std::string & query_id,
DB::ThreadStatus * thread_ptr) const
{
DB::ThreadStatus thread_status;
/// Send logs from this thread to client if possible.
/// It will allow client to see failure messages directly.
if (thread_ptr)
{
if (auto logs_queue = thread_ptr->getInternalTextLogsQueue())
DB::CurrentThread::attachInternalTextLogsQueue(logs_queue, DB::LogsLevel::trace);
}
LOG_FATAL(log, "########################################");
if (query_id.empty())
@ -280,6 +295,10 @@ private:
/// Write symbolized stack trace line by line for better grep-ability.
stack_trace.toStringEveryLine([&](const std::string & s) { LOG_FATAL(log, s); });
/// When everything is done, we will try to send these error messages to client.
if (thread_ptr)
thread_ptr->onFatalError();
}
};

View File

@ -1,9 +1,9 @@
# This strings autochanged from release_lib.sh:
SET(VERSION_REVISION 54435)
SET(VERSION_REVISION 54436)
SET(VERSION_MAJOR 20)
SET(VERSION_MINOR 5)
SET(VERSION_MINOR 6)
SET(VERSION_PATCH 1)
SET(VERSION_GITHASH 91df18a906dcffdbee6816e5389df6c65f86e35f)
SET(VERSION_DESCRIBE v20.5.1.1-prestable)
SET(VERSION_STRING 20.5.1.1)
SET(VERSION_GITHASH efc57fb063b3fb4df968d916720ec4d4ced4642e)
SET(VERSION_DESCRIBE v20.6.1.1-prestable)
SET(VERSION_STRING 20.6.1.1)
# end of autochange

4
debian/changelog vendored
View File

@ -1,5 +1,5 @@
clickhouse (20.5.1.1) unstable; urgency=low
clickhouse (20.6.1.1) unstable; urgency=low
* Modified source code
-- clickhouse-release <clickhouse-release@yandex-team.ru> Tue, 28 Apr 2020 20:12:13 +0300
-- clickhouse-release <clickhouse-release@yandex-team.ru> Mon, 22 Jun 2020 20:40:23 +0300

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.5.1.*
ARG version=20.6.1.*
RUN apt-get update \
&& apt-get install --yes --no-install-recommends \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.5.1.*
ARG version=20.6.1.*
ARG gosu_ver=1.10
RUN apt-get update \

View File

@ -1,7 +1,7 @@
FROM ubuntu:18.04
ARG repository="deb https://repo.clickhouse.tech/deb/stable/ main/"
ARG version=20.5.1.*
ARG version=20.6.1.*
RUN apt-get update && \
apt-get install -y apt-transport-https dirmngr && \

View File

@ -7,3 +7,7 @@ services:
POSTGRES_PASSWORD: mysecretpassword
ports:
- 5432:5432
networks:
default:
aliases:
- postgre-sql.local

View File

@ -5,4 +5,11 @@ toc_priority: 25
toc_title: hidden
---
# ClickHouse Engines
There are two key engine kinds in ClickHouse:
- [Table engines](table-engines/index.md)
- [Database engines](database-engines/index.md)
{## [Original article](https://clickhouse.tech/docs/en/engines/) ##}

View File

@ -19,7 +19,7 @@ The table engine (type of table) determines:
### MergeTree {#mergetree}
The most universal and functional table engines for high-load tasks. The property shared by these engines is quick data insertion with subsequent background data processing. `MergeTree` family engines support data replication (with [Replicated\*](../../engines/table-engines/mergetree-family/replication.md#table_engines-replication) versions of engines), partitioning, and other features not supported in other engines.
The most universal and functional table engines for high-load tasks. The property shared by these engines is quick data insertion with subsequent background data processing. `MergeTree` family engines support data replication (with [Replicated\*](../../engines/table-engines/mergetree-family/replication.md#table_engines-replication) versions of engines), partitioning, secondary data-skipping indexes, and other features not supported in other engines.
Engines in the family:
@ -80,4 +80,4 @@ To select data from a virtual column, you must specify its name in the `SELECT`
If you create a table with a column that has the same name as one of the table virtual columns, the virtual column becomes inaccessible. We dont recommend doing this. To help avoid conflicts, virtual column names are usually prefixed with an underscore.
[Original article](https://clickhouse.tech/docs/en/operations/table_engines/) <!--hide-->
[Original article](https://clickhouse.tech/docs/en/engines/table-engines/) <!--hide-->

View File

@ -821,6 +821,10 @@ ClickHouse supports the following algorithms of choosing replicas:
- [First or random](#load_balancing-first_or_random)
- [Round robin](#load_balancing-round_robin)
See also:
- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors)
### Random (by Default) {#load_balancing-random}
``` sql
@ -1170,8 +1174,10 @@ Controls how fast errors in distributed tables are zeroed. If a replica is unava
See also:
- [load\_balancing](#load_balancing-round_robin)
- [Table engine Distributed](../../engines/table-engines/special/distributed.md)
- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap)
- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors)
## distributed\_replica\_error\_cap {#settings-distributed_replica_error_cap}
@ -1182,8 +1188,24 @@ Error count of each replica is capped at this value, preventing a single replica
See also:
- [load\_balancing](#load_balancing-round_robin)
- [Table engine Distributed](../../engines/table-engines/special/distributed.md)
- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life)
- [distributed\_replica\_max\_ignored\_errors](#settings-distributed_replica_max_ignored_errors)
## distributed\_replica\_max\_ignored\_errors {#settings-distributed_replica_max_ignored_errors}
- Type: unsigned int
- Default value: 0
Number of errors that will be ignored while choosing replicas (according to `load_balancing` algorithm).
See also:
- [load\_balancing](#load_balancing-round_robin)
- [Table engine Distributed](../../engines/table-engines/special/distributed.md)
- [distributed\_replica\_error\_cap](#settings-distributed_replica_error_cap)
- [distributed\_replica\_error\_half\_life](#settings-distributed_replica_error_half_life)
## distributed\_directory\_monitor\_sleep\_time\_ms {#distributed_directory_monitor_sleep_time_ms}

View File

@ -9,6 +9,7 @@ The following aggregate functions are supported:
- [`min`](../../sql-reference/aggregate-functions/reference/min.md#agg_function-min)
- [`max`](../../sql-reference/aggregate-functions/reference/max.md#agg_function-max)
- [`sum`](../../sql-reference/aggregate-functions/reference/sum.md#agg_function-sum)
- [`sumWithOverflow`](../../sql-reference/aggregate-functions/reference/sumwithoverflow.md#sumwithoverflowx)
- [`groupBitAnd`](../../sql-reference/aggregate-functions/reference/groupbitand.md#groupbitand)
- [`groupBitOr`](../../sql-reference/aggregate-functions/reference/groupbitor.md#groupbitor)
- [`groupBitXor`](../../sql-reference/aggregate-functions/reference/groupbitxor.md#groupbitxor)

View File

@ -276,7 +276,7 @@ $ curl -sS 'http://localhost:8123/?max_result_bytes=4000000&buffer_size=3000000&
### Пример {#primer}
``` bash
$ curl -sS "<address>?param_id=2¶m_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}"
$ curl -sS "http://localhost:8123/?param_id=2&param_phrase=test" -d "SELECT * FROM table WHERE int_column = {id:UInt8} and string_column = {phrase:String}"
```
## Предопределенный HTTP интерфейс {#predefined_http_interface}

View File

@ -123,7 +123,7 @@ private:
};
bool is_interactive = true; /// Use either interactive line editing interface or batch mode.
bool need_render_progress = true; /// Render query execution progress.
bool send_logs = false; /// send_logs_level passed, do not use previous cursor position, to avoid overlaps with logs
bool has_received_logs = false; /// We have received some logs, do not use previous cursor position, to avoid overlaps with logs
bool echo_queries = false; /// Print queries before execution in batch mode.
bool ignore_error = false; /// In case of errors, don't print error message, continue to next query. Only applicable for non-interactive mode.
bool print_time_to_stderr = false; /// Output execution time to stderr in batch mode.
@ -908,8 +908,6 @@ private:
connection->forceConnected(connection_parameters.timeouts);
send_logs = context.getSettingsRef().send_logs_level != LogsLevel::none;
ASTPtr input_function;
if (insert && insert->select)
insert->tryFindInputFunction(input_function);
@ -1518,6 +1516,7 @@ private:
void onLogData(Block & block)
{
has_received_logs = true;
initLogsOutputStream();
logs_out_stream->write(block);
logs_out_stream->flush();
@ -1553,7 +1552,7 @@ private:
void clearProgress()
{
written_progress_chars = 0;
if (!send_logs)
if (!has_received_logs)
std::cerr << "\r" CLEAR_TO_END_OF_LINE;
}
@ -1581,7 +1580,7 @@ private:
const char * indicator = indicators[increment % 8];
if (!send_logs && written_progress_chars)
if (!has_received_logs && written_progress_chars)
message << '\r';
size_t prefix_size = message.count();
@ -1635,7 +1634,7 @@ private:
message << CLEAR_TO_END_OF_LINE;
if (send_logs)
if (has_received_logs)
message << '\n';
++increment;

View File

@ -65,7 +65,11 @@ std::string validateODBCConnectionString(const std::string & connection_string)
else
throw Exception("ODBC connection string parameter name doesn't begin with valid identifier character", ErrorCodes::BAD_ODBC_CONNECTION_STRING);
while (pos < end && isWordCharASCII(*pos))
/// Additionally allow dash and dot symbols in names.
/// Strictly speaking, the name with that characters should be escaped.
/// But some ODBC drivers (e.g.) Postgres don't like escaping.
while (pos < end && (isWordCharASCII(*pos) || *pos == '-' || *pos == '.'))
++pos;
return std::string(begin, pos);
@ -213,7 +217,11 @@ std::string validateODBCConnectionString(const std::string & connection_string)
auto write_value = [&](const std::string & value)
{
if (std::all_of(value.begin(), value.end(), isWordCharASCII))
/// Additionally allow dash and dot symbols - for hostnames.
/// Strictly speaking, hostname with that characters should be escaped.
/// But some ODBC drivers (e.g.) Postgres don't like escaping.
if (std::all_of(value.begin(), value.end(), [](char c) { return isWordCharASCII(c) || c == '.' || c == '-'; }))
write_plain_value(value);
else
write_escaped_value(value);

View File

@ -84,7 +84,10 @@ IConnectionPool::Entry ConnectionPoolWithFailover::get(const ConnectionTimeouts
break;
}
return Base::get(try_get_entry, get_priority);
UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0;
bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true;
return Base::get(max_ignored_errors, fallback_to_stale_replicas, try_get_entry, get_priority);
}
ConnectionPoolWithFailover::Status ConnectionPoolWithFailover::getStatus() const
@ -206,9 +209,12 @@ std::vector<ConnectionPoolWithFailover::TryResult> ConnectionPoolWithFailover::g
break;
}
bool fallback_to_stale_replicas = settings ? bool(settings->fallback_to_stale_replicas_for_distributed_queries) : true;
UInt64 max_ignored_errors = settings ? settings->distributed_replica_max_ignored_errors.value : 0;
bool fallback_to_stale_replicas = settings ? settings->fallback_to_stale_replicas_for_distributed_queries.value : true;
return Base::getMany(min_entries, max_entries, max_tries, try_get_entry, get_priority, fallback_to_stale_replicas);
return Base::getMany(min_entries, max_entries, max_tries,
max_ignored_errors, fallback_to_stale_replicas,
try_get_entry, get_priority);
}
ConnectionPoolWithFailover::TryResult

View File

@ -72,6 +72,13 @@ void CurrentThread::attachInternalTextLogsQueue(const std::shared_ptr<InternalTe
current_thread->attachInternalTextLogsQueue(logs_queue, client_logs_level);
}
void CurrentThread::setFatalErrorCallback(std::function<void()> callback)
{
if (unlikely(!current_thread))
return;
current_thread->setFatalErrorCallback(callback);
}
std::shared_ptr<InternalTextLogsQueue> CurrentThread::getInternalTextLogsQueue()
{
/// NOTE: this method could be called at early server startup stage

View File

@ -46,6 +46,8 @@ public:
LogsLevel client_logs_level);
static std::shared_ptr<InternalTextLogsQueue> getInternalTextLogsQueue();
static void setFatalErrorCallback(std::function<void()> callback);
/// Makes system calls to update ProfileEvents that contain info from rusage and taskstats
static void updatePerformanceCounters();

View File

@ -100,28 +100,28 @@ public:
/// this functor. The pools with lower result value will be tried first.
using GetPriorityFunc = std::function<size_t(size_t index)>;
/// Returns a single connection.
Entry get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc());
/// Returns at least min_entries and at most max_entries connections (at most one connection per nested pool).
/// The method will throw if it is unable to get min_entries alive connections or
/// if fallback_to_stale_replicas is false and it is unable to get min_entries connections to up-to-date replicas.
std::vector<TryResult> getMany(
size_t min_entries, size_t max_entries, size_t max_tries,
size_t max_ignored_errors,
bool fallback_to_stale_replicas,
const TryGetEntryFunc & try_get_entry,
const GetPriorityFunc & get_priority = GetPriorityFunc(),
bool fallback_to_stale_replicas = true);
void reportError(const Entry & entry);
const GetPriorityFunc & get_priority = GetPriorityFunc());
protected:
struct PoolState;
using PoolStates = std::vector<PoolState>;
/// Returns a single connection.
Entry get(size_t max_ignored_errors, bool fallback_to_stale_replicas,
const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority = GetPriorityFunc());
/// This function returns a copy of pool states to avoid race conditions when modifying shared pool states.
PoolStates updatePoolStates();
PoolStates updatePoolStates(size_t max_ignored_errors);
PoolStates getPoolStates() const;
NestedPools nested_pools;
@ -139,9 +139,13 @@ protected:
template <typename TNestedPool>
typename TNestedPool::Entry
PoolWithFailoverBase<TNestedPool>::get(const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority)
PoolWithFailoverBase<TNestedPool>::get(size_t max_ignored_errors, bool fallback_to_stale_replicas,
const TryGetEntryFunc & try_get_entry, const GetPriorityFunc & get_priority)
{
std::vector<TryResult> results = getMany(1, 1, 1, try_get_entry, get_priority);
std::vector<TryResult> results = getMany(
1 /* min entries */, 1 /* max entries */, 1 /* max tries */,
max_ignored_errors, fallback_to_stale_replicas,
try_get_entry, get_priority);
if (results.empty() || results[0].entry.isNull())
throw DB::Exception(
"PoolWithFailoverBase::getMany() returned less than min_entries entries.",
@ -153,12 +157,13 @@ template <typename TNestedPool>
std::vector<typename PoolWithFailoverBase<TNestedPool>::TryResult>
PoolWithFailoverBase<TNestedPool>::getMany(
size_t min_entries, size_t max_entries, size_t max_tries,
size_t max_ignored_errors,
bool fallback_to_stale_replicas,
const TryGetEntryFunc & try_get_entry,
const GetPriorityFunc & get_priority,
bool fallback_to_stale_replicas)
const GetPriorityFunc & get_priority)
{
/// Update random numbers and error counts.
PoolStates pool_states = updatePoolStates();
PoolStates pool_states = updatePoolStates(max_ignored_errors);
if (get_priority)
{
for (size_t i = 0; i < pool_states.size(); ++i)
@ -295,22 +300,6 @@ PoolWithFailoverBase<TNestedPool>::getMany(
return try_results;
}
template <typename TNestedPool>
void PoolWithFailoverBase<TNestedPool>::reportError(const Entry & entry)
{
for (size_t i = 0; i < nested_pools.size(); ++i)
{
if (nested_pools[i]->contains(entry))
{
std::lock_guard lock(pool_states_mutex);
auto & pool_state = shared_pool_states[i];
pool_state.error_count = std::min(max_error_cap, pool_state.error_count + 1);
return;
}
}
throw DB::Exception("Can't find pool to report error", DB::ErrorCodes::LOGICAL_ERROR);
}
template <typename TNestedPool>
struct PoolWithFailoverBase<TNestedPool>::PoolState
{
@ -335,7 +324,7 @@ private:
template <typename TNestedPool>
typename PoolWithFailoverBase<TNestedPool>::PoolStates
PoolWithFailoverBase<TNestedPool>::updatePoolStates()
PoolWithFailoverBase<TNestedPool>::updatePoolStates(size_t max_ignored_errors)
{
PoolStates result;
result.reserve(nested_pools.size());
@ -354,14 +343,17 @@ PoolWithFailoverBase<TNestedPool>::updatePoolStates()
if (delta >= 0)
{
const UInt64 MAX_BITS = sizeof(UInt64) * CHAR_BIT;
size_t shift_amount = MAX_BITS;
/// Divide error counts by 2 every decrease_error_period seconds.
size_t shift_amount = delta / decrease_error_period;
if (decrease_error_period)
shift_amount = delta / decrease_error_period;
/// Update time but don't do it more often than once a period.
/// Else if the function is called often enough, error count will never decrease.
if (shift_amount)
last_error_decrease_time = current_time;
if (shift_amount >= sizeof(UInt64) * CHAR_BIT)
if (shift_amount >= MAX_BITS)
{
for (auto & state : shared_pool_states)
state.error_count = 0;
@ -378,6 +370,11 @@ PoolWithFailoverBase<TNestedPool>::updatePoolStates()
result.assign(shared_pool_states.begin(), shared_pool_states.end());
}
/// distributed_replica_max_ignored_errors
for (auto & state : result)
state.error_count = std::max<UInt64>(0, state.error_count - max_ignored_errors);
return result;
}

View File

@ -98,4 +98,15 @@ void ThreadStatus::attachInternalTextLogsQueue(const InternalTextLogsQueuePtr &
thread_group->client_logs_level = client_logs_level;
}
void ThreadStatus::setFatalErrorCallback(std::function<void()> callback)
{
fatal_error_callback = std::move(callback);
}
void ThreadStatus::onFatalError()
{
if (fatal_error_callback)
fatal_error_callback();
}
}

View File

@ -145,6 +145,10 @@ public:
void attachInternalTextLogsQueue(const InternalTextLogsQueuePtr & logs_queue,
LogsLevel client_logs_level);
/// Callback that is used to trigger sending fatal error messages to client.
void setFatalErrorCallback(std::function<void()> callback);
void onFatalError();
/// Sets query context for current thread and its thread group
/// NOTE: query_context have to be alive until detachQuery() is called
void attachQueryContext(Context & query_context);
@ -200,6 +204,9 @@ protected:
std::unique_ptr<RUsageCounters> last_rusage;
std::unique_ptr<TasksStatsCounters> taskstats;
/// Is used to send logs from logs_queue to client in case of fatal errors.
std::function<void()> fatal_error_callback;
private:
void setupState(const ThreadGroupStatusPtr & thread_group_);
};

View File

@ -316,7 +316,7 @@ struct Settings : public SettingsCollection<Settings>
M(SettingBool, log_profile_events, true, "Log query performance statistics into the query_log and query_thread_log.", 0) \
M(SettingBool, log_query_settings, true, "Log query settings into the query_log.", 0) \
M(SettingBool, log_query_threads, true, "Log query threads into system.query_thread_log table. This setting have effect only when 'log_queries' is true.", 0) \
M(SettingLogsLevel, send_logs_level, LogsLevel::none, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'none'", 0) \
M(SettingLogsLevel, send_logs_level, LogsLevel::fatal, "Send server text logs with specified minimum level to client. Valid values: 'trace', 'debug', 'information', 'warning', 'error', 'fatal', 'none'", 0) \
M(SettingBool, enable_optimize_predicate_expression, 1, "If it is set to true, optimize predicates to subqueries.", 0) \
M(SettingBool, enable_optimize_predicate_expression_to_final_subquery, 1, "Allow push predicate to final subquery.", 0) \
\
@ -348,6 +348,7 @@ struct Settings : public SettingsCollection<Settings>
\
M(SettingSeconds, distributed_replica_error_half_life, DBMS_CONNECTION_POOL_WITH_FAILOVER_DEFAULT_DECREASE_ERROR_PERIOD, "Time period reduces replica error counter by 2 times.", 0) \
M(SettingUInt64, distributed_replica_error_cap, DBMS_CONNECTION_POOL_WITH_FAILOVER_MAX_ERROR_COUNT, "Max number of errors per replica, prevents piling up an incredible amount of errors if replica was offline for some time and allows it to be reconsidered in a shorter amount of time.", 0) \
M(SettingUInt64, distributed_replica_max_ignored_errors, 0, "Number of errors that will be ignored while choosing replicas", 0) \
\
M(SettingBool, allow_experimental_live_view, false, "Enable LIVE VIEW. Not mature enough.", 0) \
M(SettingSeconds, live_view_heartbeat_interval, DEFAULT_LIVE_VIEW_HEARTBEAT_INTERVAL_SEC, "The heartbeat interval in seconds to indicate live query is alive.", 0) \

View File

@ -542,6 +542,7 @@ IMPLEMENT_SETTING_ENUM(FormatSettings::DateTimeInputFormat, DATE_TIME_INPUT_FORM
#define LOGS_LEVEL_LIST_OF_NAMES(M) \
M(none, "none") \
M(fatal, "fatal") \
M(error, "error") \
M(warning, "warning") \
M(information, "information") \

View File

@ -302,6 +302,7 @@ using SettingDateTimeInputFormat = SettingEnum<FormatSettings::DateTimeInputForm
enum class LogsLevel
{
none = 0, /// Disable
fatal,
error,
warning,
information,

View File

@ -30,7 +30,9 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
static const std::vector<String> supported_functions{"any", "anyLast", "min", "max", "sum", "groupBitAnd", "groupBitOr", "groupBitXor", "sumMap", "groupArrayArray", "groupUniqArrayArray"};
static const std::vector<String> supported_functions{"any", "anyLast", "min",
"max", "sum", "sumWithOverflow", "groupBitAnd", "groupBitOr", "groupBitXor",
"sumMap", "groupArrayArray", "groupUniqArrayArray"};
String DataTypeCustomSimpleAggregateFunction::getName() const

View File

@ -65,6 +65,21 @@ void registerDataTypeNumbers(DataTypeFactory & factory)
factory.registerAlias("DOUBLE", "Float64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("DOUBLE PRECISION", "Float64", DataTypeFactory::CaseInsensitive);
/// MySQL
factory.registerAlias("TINYINT SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1 SIGNED", "Int8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT SIGNED", "Int16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER SIGNED", "Int32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT SIGNED", "Int64", DataTypeFactory::CaseInsensitive);
factory.registerAlias("TINYINT UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT1 UNSIGNED", "UInt8", DataTypeFactory::CaseInsensitive);
factory.registerAlias("SMALLINT UNSIGNED", "UInt16", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INT UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("INTEGER UNSIGNED", "UInt32", DataTypeFactory::CaseInsensitive);
factory.registerAlias("BIGINT UNSIGNED", "UInt64", DataTypeFactory::CaseInsensitive);
}
}

View File

@ -55,6 +55,14 @@ bool ParserDataType::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
if (ParserKeyword("PRECISION").ignore(pos))
type_name_suffix = "PRECISION";
}
else if (type_name_upper.find("INT") != std::string::npos)
{
/// Support SIGNED and UNSIGNED integer type modifiers for compatibility with MySQL
if (ParserKeyword("SIGNED").ignore(pos))
type_name_suffix = "SIGNED";
else if (ParserKeyword("UNSIGNED").ignore(pos))
type_name_suffix = "UNSIGNED";
}
if (!type_name_suffix.empty())
type_name = type_name_upper + " " + type_name_suffix;

View File

@ -232,15 +232,12 @@ HTTPHandler::HTTPHandler(IServer & server_, const std::string & name)
void HTTPHandler::processQuery(
Context & context,
Poco::Net::HTTPServerRequest & request,
HTMLForm & params,
Poco::Net::HTTPServerResponse & response,
Output & used_output)
{
Context context = server.context();
CurrentThread::QueryScope query_scope(context);
LOG_TRACE(log, "Request URI: {}", request.getURI());
std::istream & istr = request.stream();
@ -683,6 +680,11 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
setThreadName("HTTPHandler");
ThreadStatus thread_status;
/// Should be initialized before anything,
/// For correct memory accounting.
Context context = server.context();
CurrentThread::QueryScope query_scope(context);
Output used_output;
/// In case of exception, send stack trace to client.
@ -706,7 +708,7 @@ void HTTPHandler::handleRequest(Poco::Net::HTTPServerRequest & request, Poco::Ne
throw Exception("The Transfer-Encoding is not chunked and there is no Content-Length header for POST request", ErrorCodes::HTTP_LENGTH_REQUIRED);
}
processQuery(request, params, response, used_output);
processQuery(context, request, params, response, used_output);
LOG_INFO(log, "Done processing query");
}
catch (...)

View File

@ -72,6 +72,7 @@ private:
/// Also initializes 'used_output'.
void processQuery(
Context & context,
Poco::Net::HTTPServerRequest & request,
HTMLForm & params,
Poco::Net::HTTPServerResponse & response,

View File

@ -189,6 +189,7 @@ void TCPHandler::runImpl()
state.logs_queue = std::make_shared<InternalTextLogsQueue>();
state.logs_queue->max_priority = Poco::Logger::parseLevel(client_logs_level.toString());
CurrentThread::attachInternalTextLogsQueue(state.logs_queue, client_logs_level);
CurrentThread::setFatalErrorCallback([this]{ sendLogs(); });
}
query_context->setExternalTablesInitializer([&connection_settings, this] (Context & context)

View File

@ -321,7 +321,6 @@ void MergeTreeBaseSelectProcessor::executePrewhereActions(Block & block, const P
else
{
auto & ctn = block.getByName(prewhere_info->prewhere_column_name);
ctn.type = std::make_shared<DataTypeUInt8>();
ctn.column = ctn.type->createColumnConst(block.rows(), 1u)->convertToFullColumnIfConst();
}

View File

@ -4,9 +4,11 @@ const char * auto_contributors[] {
"243f6a8885a308d313198a2e037",
"821008736@qq.com",
"Akazz",
"Albert Kidrachev",
"Alberto",
"Aleksandra (Ася)",
"Aleksei Levushkin",
"Aleksey",
"Aleksey Akulovich",
"Alex Bocharov",
"Alex Krash",
@ -32,12 +34,15 @@ const char * auto_contributors[] {
"Alexander Sapin",
"Alexander Tokmakov",
"Alexander Tretiakov",
"Alexandr Kondratev",
"Alexandr Krasheninnikov",
"Alexandr Orlov",
"Alexei Averchenko",
"Alexey",
"Alexey Arno",
"Alexey Dushechkin",
"Alexey Elymanov",
"Alexey Ilyukhov",
"Alexey Milovidov",
"Alexey Tronov",
"Alexey Vasiliev",
@ -53,18 +58,22 @@ const char * auto_contributors[] {
"AndreevDm",
"Andrei Bodrov",
"Andrei Chulkov",
"Andrei Nekrashevich",
"Andrew Grigorev",
"Andrew Onyshchuk",
"Andrey",
"Andrey Chulkov",
"Andrey Dudin",
"Andrey Kadochnikov",
"Andrey Konchagin",
"Andrey Konyaev",
"Andrey M",
"Andrey Mironov",
"Andrey Skobtsov",
"Andrey Urusov",
"Andy Yang",
"Anton Kobzev",
"Anton Okhitin",
"Anton Okulov",
"Anton Patsev",
"Anton Popov",
@ -74,7 +83,9 @@ const char * auto_contributors[] {
"Anton Zhabolenko",
"Arsen Hakobyan",
"Artem Andreenko",
"Artem Gavrilov",
"Artem Konovalov",
"Artem Streltsov",
"Artem Zuikov",
"Artemeey",
"Artemkin Pavel",
@ -90,6 +101,7 @@ const char * auto_contributors[] {
"BanyRule",
"Baudouin Giard",
"BayoNet",
"Bharat Nallan",
"Big Elephant",
"BlahGeek",
"Bogdan",
@ -101,6 +113,7 @@ const char * auto_contributors[] {
"Brett Hoerner",
"Bulat Gaifullin",
"Carbyn",
"Chao Wang",
"Chen Yufei",
"Ciprian Hacman",
"Clement Rodriguez",
@ -119,6 +132,7 @@ const char * auto_contributors[] {
"Denis Zhuravlev",
"Derek Perkins",
"Ding Xiang Fei",
"Dmitriev Mikhail",
"Dmitrii Kovalkov",
"Dmitrii Raev",
"Dmitry",
@ -132,6 +146,7 @@ const char * auto_contributors[] {
"Dmitry Rubashkin",
"Dmitry S..ky / skype: dvska-at-skype",
"Doge",
"DoomzD",
"Dr. Strange Looker",
"Eldar Zaitov",
"Elena Baskakova",
@ -141,17 +156,21 @@ const char * auto_contributors[] {
"Ernest Poletaev",
"Eugene Klimov",
"Eugene Konkov",
"Evgenia Sudarikova",
"Evgenii Pravda",
"Evgeniy Gatov",
"Evgeniy Udodov",
"Evgeny Konkov",
"Ewout",
"Fabian Stäber",
"Fadi Hadzh",
"Fan()",
"FawnD2",
"FeehanG",
"Filipe Caixeta",
"Flowyi",
"Francisco Barón",
"Frank Zhao",
"Fruit of Eden",
"Gagan Arneja",
"Gary Dotzler",
@ -196,6 +215,7 @@ const char * auto_contributors[] {
"Ivan Kushnarenko",
"Ivan Lezhankin",
"Ivan Remen",
"Ivan Starkov",
"Ivan Zhukov",
"JaosnHsieh",
"Jason",
@ -216,6 +236,7 @@ const char * auto_contributors[] {
"Konstantin Malanchev",
"Konstantin Podshumok",
"Korviakov Andrey",
"Kozlov Ivan",
"Kruglov Pavel",
"Leonardo Cecchi",
"Leopold Schabel",
@ -246,6 +267,7 @@ const char * auto_contributors[] {
"Max Akhmedov",
"Max Vetrov",
"Maxim Akhmedov",
"Maxim Babenko",
"Maxim Fedotov",
"Maxim Fridental",
"Maxim Khrisanfov",
@ -263,6 +285,7 @@ const char * auto_contributors[] {
"Michael Kolupaev",
"Michael Monashev",
"Michael Razuvaev",
"Michael Smitasin",
"Michal Lisowski",
"Mihail Fandyushin",
"Mikahil Nacharov",
@ -271,12 +294,14 @@ const char * auto_contributors[] {
"Mikhail Fandyushin",
"Mikhail Filimonov",
"Mikhail Korotov",
"Mikhail Malafeev",
"Mikhail Nacharov",
"Mikhail Salosin",
"Mikhail Surin",
"Mikhail f. Shiryaev",
"Milad Arabi",
"Mohammad Hossein Sekhavat",
"MovElb",
"Murat Kabilov",
"NIKITA MIKHAILOV",
"Narek Galstyan",
@ -289,8 +314,10 @@ const char * auto_contributors[] {
"Nikita Lapkov",
"Nikita Mikhailov",
"Nikita Mikhaylov",
"Nikita Orlov",
"Nikita Vasilev",
"Nikolai Kochetov",
"Nikolai Sorokin",
"Nikolay Degterinsky",
"Nikolay Kirsh",
"Nikolay Semyachkin",
@ -305,6 +332,7 @@ const char * auto_contributors[] {
"Olga Khvostikova",
"Orivej Desh",
"Oskar Wojciski",
"Paramtamtam",
"Pavel",
"Pavel Kartaviy",
"Pavel Kartavyy",
@ -315,8 +343,10 @@ const char * auto_contributors[] {
"Pavlo Bashynskiy",
"Pawel Rog",
"Persiyanov Dmitriy Andreevich",
"Pervakov Grigorii",
"Pervakov Grigory",
"Philippe Ombredanne",
"Potya",
"Pradeep Chhetri",
"Quid37",
"Rafael David Tinoco",
@ -324,6 +354,7 @@ const char * auto_contributors[] {
"Ravengg",
"Reilee",
"Reto Kromer",
"Ri",
"Roman Lipovsky",
"Roman Nikolaev",
"Roman Nozdrin",
@ -331,10 +362,12 @@ const char * auto_contributors[] {
"Roman Tsisyk",
"Ruslan",
"Ruzal Ibragimov",
"S.M.A. Djawadi",
"Sabyanin Maxim",
"SaltTan",
"Sami Kerola",
"Samuel Chou",
"Saulius Valatka",
"Serge Rider",
"Sergei Bocharov",
"Sergei Semin",
@ -364,10 +397,12 @@ const char * auto_contributors[] {
"Sébastien Launay",
"TAC",
"TCeason",
"Tagir Kuskarov",
"Tangaev",
"Tema Novikov",
"The-Alchemist",
"Tobias Adamson",
"Tom Bombadil",
"Tsarkova Anastasia",
"Ubuntu",
"Ubus",
@ -377,6 +412,7 @@ const char * auto_contributors[] {
"Vadim Skipin",
"VadimPE",
"Valera Ryaboshapko",
"Vasily Morozov",
"Vasily Nemkov",
"Vasily Okunev",
"Vasily Vasilkov",
@ -396,7 +432,9 @@ const char * auto_contributors[] {
"Vivien Maisonneuve",
"Vlad Arkhipov",
"Vladimir",
"Vladimir Bunchuk",
"Vladimir Chebotarev",
"Vladimir Golovchenko",
"Vladimir Goncharov",
"Vladimir Kolobaev",
"Vladimir Kopysov",
@ -405,6 +443,7 @@ const char * auto_contributors[] {
"Vladislav Rassokhin",
"Vladislav Smirnov",
"Vojtech Splichal",
"Volodymyr Kuznetsov",
"Vsevolod Orlov",
"Vxider",
"Vyacheslav Alipov",
@ -416,6 +455,7 @@ const char * auto_contributors[] {
"Yegor Andreenko",
"Yiğit Konur",
"Yohann Jardin",
"Yuntao Wu",
"Yuri Dyachenko",
"Yurii Vlasenko",
"Yuriy",
@ -423,6 +463,7 @@ const char * auto_contributors[] {
"Yury Karpovich",
"Yury Stankevich",
"Zhichang Yu",
"Zhipeng",
"abdrakhmanov",
"abyss7",
"achimbab",
@ -448,6 +489,8 @@ const char * auto_contributors[] {
"benamazing",
"bgranvea",
"blazerer",
"bluebirddm",
"bobrovskij artemij",
"bseng",
"cekc",
"champtar",
@ -462,6 +505,7 @@ const char * auto_contributors[] {
"decaseal",
"dependabot-preview[bot]",
"dependabot[bot]",
"dgrr",
"dimarub2000",
"dinosaur",
"dmitrii",
@ -471,20 +515,28 @@ const char * auto_contributors[] {
"egatov",
"elBroom",
"elenaspb2019",
"emironyuk",
"evtan",
"exprmntr",
"ezhaka",
"f1yegor",
"favstovol",
"felixoid",
"fenglv",
"fessmage",
"filimonov",
"flow",
"foxxmary",
"frank",
"franklee",
"fredchenbj",
"g-arslan",
"ggerogery",
"giordyb",
"glockbender",
"hao.he",
"hcz",
"hexiaoting",
"hotid",
"igor",
"igor.lapko",
@ -500,6 +552,7 @@ const char * auto_contributors[] {
"kreuzerkrieg",
"ks1322",
"kshvakov",
"kssenii",
"l",
"lalex",
"leozhang",
@ -513,11 +566,14 @@ const char * auto_contributors[] {
"liuyimin",
"liyang",
"lomberts",
"long2ice",
"luc1ph3r",
"madianjun",
"maiha",
"malkfilipp",
"maqroll",
"maxim",
"maxim-babenko",
"maxkuzn",
"maxulan",
"memo",
@ -527,8 +583,10 @@ const char * auto_contributors[] {
"mfridental",
"miha-g",
"millb",
"mnkonkova",
"morty",
"moscas",
"myrrc",
"nagorny",
"never lee",
"nicelulu",
@ -543,6 +601,7 @@ const char * auto_contributors[] {
"palasonicq",
"peshkurov",
"philip.han",
"potya",
"proller",
"pufit",
"pyos",
@ -562,6 +621,7 @@ const char * auto_contributors[] {
"simon-says",
"spyros87",
"stavrolia",
"stepenhu",
"sundy",
"sundy-li",
"sundyli",
@ -577,9 +637,11 @@ const char * auto_contributors[] {
"velom",
"vicdashkov",
"vinity",
"vitstn",
"vxider",
"vzakaznikov",
"wangchao",
"xPoSx",
"yonesko",
"zamulla",
"zhang2014",
@ -600,7 +662,9 @@ const char * auto_contributors[] {
"小路",
"张健",
"张风啸",
"极客青年",
"谢磊",
"黄朝晖",
"黄璞",
"박현우",
nullptr};

View File

@ -271,6 +271,10 @@ def run_tests_array(all_tests_with_params):
if stderr:
print(stderr.encode('utf-8'))
# Stop on fatal errors like segmentation fault. They are send to client via logs.
if ' <Fatal> ' in stderr:
SERVER_DIED = True
if args.stop and ('Connection refused' in stderr or 'Attempt to read after eof' in stderr) and not 'Received exception from server' in stderr:
SERVER_DIED = True

View File

@ -732,6 +732,10 @@ class ClickHouseInstance:
self.ipv6_address = ipv6_address
self.with_installed_binary = with_installed_binary
def is_built_with_thread_sanitizer(self):
build_opts = self.query("SELECT value FROM system.build_options WHERE name = 'CXX_FLAGS'")
return "-fsanitize=thread" in build_opts
# Connects to the instance via clickhouse-client, sends a query (1st argument) and returns the answer
def query(self, sql, stdin=None, timeout=None, settings=None, user=None, password=None, ignore_error=False):
return self.client.query(sql, stdin, timeout, settings, user, password, ignore_error)

View File

@ -153,6 +153,7 @@ def get_dict(source, layout, fields, suffix_name=''):
dictionary.generate_config()
return dictionary
def setup_module(module):
global DICTIONARIES
global cluster
@ -210,8 +211,42 @@ def get_dictionaries(fold, total_folds, all_dicts):
return all_dicts[fold * chunk_len : (fold + 1) * chunk_len]
def remove_mysql_dicts():
"""
We have false-positive race condition in our openSSL version.
MySQL dictionary use OpenSSL, so to prevent known failure we
disable tests for these dictionaries.
Read of size 8 at 0x7b3c00005dd0 by thread T61 (mutexes: write M1010349240585225536):
#0 EVP_CIPHER_mode <null> (clickhouse+0x13b2223b)
#1 do_ssl3_write <null> (clickhouse+0x13a137bc)
#2 ssl3_write_bytes <null> (clickhouse+0x13a12387)
#3 ssl3_write <null> (clickhouse+0x139db0e6)
#4 ssl_write_internal <null> (clickhouse+0x139eddce)
#5 SSL_write <null> (clickhouse+0x139edf20)
#6 ma_tls_write <null> (clickhouse+0x139c7557)
#7 ma_pvio_tls_write <null> (clickhouse+0x139a8f59)
#8 ma_pvio_write <null> (clickhouse+0x139a8488)
#9 ma_net_real_write <null> (clickhouse+0x139a4e2c)
#10 ma_net_write_command <null> (clickhouse+0x139a546d)
#11 mthd_my_send_cmd <null> (clickhouse+0x13992546)
#12 mysql_close_slow_part <null> (clickhouse+0x13999afd)
#13 mysql_close <null> (clickhouse+0x13999071)
#14 mysqlxx::Connection::~Connection() <null> (clickhouse+0x1370f814)
#15 mysqlxx::Pool::~Pool() <null> (clickhouse+0x13715a7b)
TODO remove this when open ssl will be fixed or thread sanitizer will be suppressed
"""
global DICTIONARIES
DICTIONARIES = [d for d in DICTIONARIES if not d.name.startswith("MySQL")]
@pytest.mark.parametrize("fold", list(range(10)))
def test_simple_dictionaries(started_cluster, fold):
if node.is_built_with_thread_sanitizer():
remove_mysql_dicts()
fields = FIELDS["simple"]
values = VALUES["simple"]
data = [Row(fields, vals) for vals in values]
@ -259,6 +294,10 @@ def test_simple_dictionaries(started_cluster, fold):
@pytest.mark.parametrize("fold", list(range(10)))
def test_complex_dictionaries(started_cluster, fold):
if node.is_built_with_thread_sanitizer():
remove_mysql_dicts()
fields = FIELDS["complex"]
values = VALUES["complex"]
data = [Row(fields, vals) for vals in values]
@ -292,6 +331,9 @@ def test_complex_dictionaries(started_cluster, fold):
@pytest.mark.parametrize("fold", list(range(10)))
def test_ranged_dictionaries(started_cluster, fold):
if node.is_built_with_thread_sanitizer():
remove_mysql_dicts()
fields = FIELDS["ranged"]
values = VALUES["ranged"]
data = [Row(fields, vals) for vals in values]
@ -380,7 +422,7 @@ def test_key_value_complex_dictionaries(started_cluster, fold):
values = VALUES["complex"]
data = [Row(fields, vals) for vals in values]
all_complex_dicts = [d for d in DICTIONARIES if d.structure.layout.layout_type == "complex"]
all_complex_dicts = [d for d in DICTIONARIES_KV if d.structure.layout.layout_type == "complex"]
complex_dicts = get_dictionaries(fold, 10, all_complex_dicts)
for dct in complex_dicts:
dct.load_data(data)

View File

@ -16,8 +16,23 @@ n3 = cluster.add_instance('n3', main_configs=['configs/remote_servers.xml'])
nodes = len(cluster.instances)
queries = nodes*5
def create_tables():
def bootstrap():
for n in cluster.instances.values():
# At startup, server loads configuration files.
#
# However ConfigReloader does not know about already loaded files
# (files is empty()), hence it will always reload the configuration
# just after server starts (+ 2 seconds, reload timeout).
#
# And on configuration reload the clusters will be re-created, so some
# internal stuff will be reseted:
# - error_count
# - last_used (round_robing)
#
# And if the reload will happen during round_robin test it will start
# querying from the beginning, so let's issue config reload just after
# start to avoid reload in the middle of the test execution.
n.query('SYSTEM RELOAD CONFIG')
n.query('DROP TABLE IF EXISTS data')
n.query('DROP TABLE IF EXISTS dist')
n.query('CREATE TABLE data (key Int) Engine=Memory()')
@ -36,7 +51,7 @@ def make_uuid():
def start_cluster():
try:
cluster.start()
create_tables()
bootstrap()
yield cluster
finally:
cluster.shutdown()
@ -112,3 +127,39 @@ def test_load_balancing_round_robin():
unique_nodes.add(get_node(n1, settings={'load_balancing': 'round_robin'}))
assert len(unique_nodes) == nodes, unique_nodes
assert unique_nodes == set(['n1', 'n2', 'n3'])
def test_distributed_replica_max_ignored_errors():
settings = {
'load_balancing': 'in_order',
'prefer_localhost_replica': 0,
'connect_timeout': 2,
'receive_timeout': 2,
'send_timeout': 2,
'idle_connection_timeout': 2,
'tcp_keep_alive_timeout': 2,
'distributed_replica_max_ignored_errors': 0,
'distributed_replica_error_half_life': 60,
}
# initiate connection (if started only this test)
n2.query('SELECT * FROM dist', settings=settings)
cluster.pause_container('n1')
# n1 paused -- skipping, and increment error_count for n1
# but the query succeeds, no need in query_and_get_error()
n2.query('SELECT * FROM dist', settings=settings)
# XXX: due to config reloading we need second time (sigh)
n2.query('SELECT * FROM dist', settings=settings)
# check error_count for n1
assert int(n2.query("""
SELECT errors_count FROM system.clusters
WHERE cluster = 'replicas_cluster' AND host_name = 'n1'
""", settings=settings)) == 1
cluster.unpause_container('n1')
# still n2
assert get_node(n2, settings=settings) == 'n2'
# now n1
settings['distributed_replica_max_ignored_errors'] = 1
assert get_node(n2, settings=settings) == 'n1'

View File

@ -81,7 +81,10 @@ class TestLiveViewOverDistributedSuite:
client1.expect(prompt)
client1.send(select_query)
client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n')
client1.expect('"node1",0,0')
client1.expect('"node1",1,1')
client1.expect('"node2",0,10')
client1.expect('"node2",1,11')
client1.expect(prompt)
client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)")
@ -90,7 +93,13 @@ class TestLiveViewOverDistributedSuite:
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect('"node1",0,0\r\n.*"node1",1,1\r\n.*"node1",1,3\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n.*"node2",0,10\r\n.*"node2",1,11\r\n')
client1.expect('"node1",0,0')
client1.expect('"node1",1,1')
client1.expect('"node1",1,3')
client1.expect('"node1",2,3')
client1.expect('"node1",3,3')
client1.expect('"node2",0,10')
client1.expect('"node2",1,11')
client1.expect(prompt)
def test_distributed_over_live_view_order_by_key(self, started_cluster, node, source):
@ -110,7 +119,10 @@ class TestLiveViewOverDistributedSuite:
client1.expect(prompt)
client1.send(select_query)
client1.expect('"node1",0,0\r\n"node2",0,10\r\n"node1",1,1\r\n.*"node2",1,11\r\n')
client1.expect('"node1",0,0')
client1.expect('"node2",0,10')
client1.expect('"node1",1,1')
client1.expect('"node2",1,11')
client1.expect(prompt)
client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 2, 3)")
@ -119,7 +131,13 @@ class TestLiveViewOverDistributedSuite:
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect('"node1",0,0\r\n.*"node2",0,10.*\r\n"node1",1,1\r\n.*"node1",1,3\r\n.*"node2",1,11\r\n.*"node1",2,3\r\n.*"node1",3,3\r\n')
client1.expect('"node1",0,0')
client1.expect('"node2",0,10')
client1.expect('"node1",1,1')
client1.expect('"node1",1,3')
client1.expect('"node2",1,11')
client1.expect('"node1",2,3')
client1.expect('"node1",3,3')
client1.expect(prompt)
def test_distributed_over_live_view_group_by_node(self, started_cluster, node, source):
@ -139,14 +157,16 @@ class TestLiveViewOverDistributedSuite:
client1.expect(prompt)
client1.send(select_query)
client1.expect('"node1",1\r\n"node2",21\r\n')
client1.expect('"node1",1')
client1.expect('"node2",21')
client1.expect(prompt)
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect('"node1",3\r\n.*"node2",21\r\n')
client1.expect('"node1",3')
client1.expect('"node2",21')
client1.expect(prompt)
client1.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)")
@ -155,7 +175,8 @@ class TestLiveViewOverDistributedSuite:
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect('"node1",12\r\n.*"node2",21\r\n')
client1.expect('"node1",12')
client1.expect('"node2",21')
client1.expect(prompt)
def test_distributed_over_live_view_group_by_key(self, started_cluster, node, source):
@ -175,21 +196,27 @@ class TestLiveViewOverDistributedSuite:
client1.expect(prompt)
client1.send(select_query)
client1.expect("0,10\r\n1,12\r\n")
client1.expect('0,10')
client1.expect('1,12')
client1.expect(prompt)
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect("0,10\r\n1,12\r\n2,2\r\n")
client1.expect('0,10')
client1.expect('1,12')
client1.expect('2,2')
client1.expect(prompt)
client2.send("INSERT INTO distributed_table VALUES ('node1', 1, 3), ('node1', 3, 3)")
client2.expect(prompt)
time.sleep(2)
client1.send(select_query)
client1.expect("0,10\r\n.*1,15\r\n.*2,2\r\n.*3,3\r\n")
client1.expect('0,10')
client1.expect('1,15')
client1.expect('2,2')
client1.expect('3,3')
client1.expect(prompt)
def test_distributed_over_live_view_sum(self, started_cluster, node, source):

View File

@ -111,15 +111,28 @@ node2\t1\t11
client1.expect(prompt)
client1.send("WATCH lv FORMAT CSV")
client1.expect('"node1",0,0,1\r\n.*"node1",1,1,1\r\n.*"node2",0,10,1\r\n.*"node2",1,11,1\r\n')
client1.expect('"node1",0,0,1')
client1.expect('"node1",1,1,1')
client1.expect('"node2",0,10,1')
client1.expect('"node2",1,11,1')
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
client1.expect('"node1",0,0,2\r\n.*"node1",1,1,2\r\n.*"node1",2,2,2\r\n.*"node2",0,10,2\r\n.*"node2",1,11,2\r\n')
client1.expect('"node1",0,0,2')
client1.expect('"node1",1,1,2')
client1.expect('"node1",2,2,2')
client1.expect('"node2",0,10,2')
client1.expect('"node2",1,11,2')
client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)")
client2.expect(prompt)
client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node1",1,1,3\r\n.*"node1",2,2,3\r\n.*"node2",0,10,3\r\n.*"node2",1,11,3\r\n.*"node3",3,3,3\r\n')
client1.expect('"node1",0,0,3')
client1.expect('"node1",0,3,3')
client1.expect('"node1",1,1,3')
client1.expect('"node1",2,2,3')
client1.expect('"node2",0,10,3')
client1.expect('"node2",1,11,3')
client1.expect('"node3",3,3,3')
def test_watch_live_view_order_by_key(self, started_cluster, node, source):
log = sys.stdout
@ -141,15 +154,28 @@ node2\t1\t11
client1.expect(prompt)
client1.send("WATCH lv FORMAT CSV")
client1.expect('"node1",0,0,1\r\n.*"node2",0,10,1\r\n.*"node1",1,1,1\r\n.*"node2",1,11,1\r\n')
client1.expect('"node1",0,0,1')
client1.expect('"node2",0,10,1')
client1.expect('"node1",1,1,1')
client1.expect('"node2",1,11,1')
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
client1.expect('"node1",0,0,2\r\n.*"node2",0,10,2\r\n.*"node1",1,1,2\r\n.*"node2",1,11,2\r\n.*"node1",2,2,2\r\n')
client1.expect('"node1",0,0,2')
client1.expect('"node2",0,10,2')
client1.expect('"node1",1,1,2')
client1.expect('"node2",1,11,2')
client1.expect('"node1",2,2,2')
client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)")
client2.expect(prompt)
client1.expect('"node1",0,0,3\r\n.*"node1",0,3,3\r\n.*"node2",0,10,3\r\n.*"node1",1,1,3\r\n.*"node2",1,11,3\r\n.*"node1",2,2,3\r\n.*"node3",3,3,3\r\n')
client1.expect('"node1",0,0,3')
client1.expect('"node1",0,3,3')
client1.expect('"node2",0,10,3')
client1.expect('"node1",1,1,3')
client1.expect('"node2",1,11,3')
client1.expect('"node1",2,2,3')
client1.expect('"node3",3,3,3')
def test_watch_live_view_group_by_node(self, started_cluster, node, source):
log = sys.stdout
@ -171,15 +197,19 @@ node2\t1\t11
client1.expect(prompt)
client1.send("WATCH lv FORMAT CSV")
client1.expect('"node1",1,1\r\n.*"node2",21,1\r\n')
client1.expect('"node1",1,1')
client1.expect('"node2",21,1')
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
client1.expect('"node1",3,2\r\n.*"node2",21,2\r\n')
client1.expect('"node1",3,2')
client1.expect('"node2",21,2')
client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node3', 3, 3)")
client2.expect(prompt)
client1.expect('"node1",6,3\r\n.*"node2",21,3\r\n.*"node3",3,3\r\n')
client1.expect('"node1",6,3')
client1.expect('"node2",21,3')
client1.expect('"node3",3,3')
def test_watch_live_view_group_by_key(self, started_cluster, node, source):
log = sys.stdout
@ -201,15 +231,21 @@ node2\t1\t11
client1.expect(prompt)
client1.send("WATCH lv FORMAT CSV")
client1.expect("0,10,1\r\n.*1,12,1\r\n")
client1.expect('0,10,1')
client1.expect('1,12,1')
client2.send("INSERT INTO distributed_table VALUES ('node1', 2, 2)")
client2.expect(prompt)
client1.expect("0,10,2\r\n.*1,12,2\r\n.*2,2,2\r\n")
client1.expect('0,10,2')
client1.expect('1,12,2')
client1.expect('2,2,2')
client2.send("INSERT INTO distributed_table VALUES ('node1', 0, 3), ('node1', 3, 3)")
client2.expect(prompt)
client1.expect("0,13,3\r\n.*1,12,3\r\n.*2,2,3\r\n.*3,3,3\r\n")
client1.expect('0,13,3')
client1.expect('1,12,3')
client1.expect('2,2,3')
client1.expect('3,3,3')
def test_watch_live_view_sum(self, started_cluster, node, source):

View File

@ -222,11 +222,16 @@ def test_postgres_odbc_hached_dictionary_no_tty_pipe_overflow(started_cluster):
def test_postgres_insert(started_cluster):
conn = get_postgres_conn()
conn.cursor().execute("truncate table clickhouse.test_table")
node1.query("create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;', 'clickhouse', 'test_table')")
# Also test with Servername containing '.' and '-' symbols (defined in
# postgres .yml file). This is needed to check parsing, validation and
# reconstruction of connection string.
node1.query("create table pg_insert (column1 UInt8, column2 String) engine=ODBC('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table')")
node1.query("insert into pg_insert values (1, 'hello'), (2, 'world')")
assert node1.query("select * from pg_insert") == '1\thello\n2\tworld\n'
node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') format CSV 3,test")
node1.query("insert into table function odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)")
node1.query("insert into table function odbc('DSN=postgresql_odbc;Servername=postgre-sql.local', 'clickhouse', 'test_table') select number, 's' || toString(number) from numbers (4, 7)")
assert node1.query("select sum(column1), count(column1) from pg_insert") == "55\t10\n"
assert node1.query("select sum(n), count(n) from (select (*,).1 as n from (select * from odbc('DSN=postgresql_odbc;', 'clickhouse', 'test_table')))") == "55\t10\n"

View File

@ -1,14 +1,13 @@
<test>
<query>SELECT max(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000000)</query>
<query>SELECT max(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000)</query>
<query>SELECT min(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000000)</query>
<query>SELECT min(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000)</query>
<query>SELECT sum(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000000)</query>
<query>SELECT sum(-1 * (((-2 * (number * -3)) * -4) * -5)) FROM numbers(500000)</query>
<query>SELECT min(-1 + (((-2 + (number + -3)) + -4) + -5)) FROM numbers(500000000)</query>
<query>SELECT min(-1 + (((-2 + (number + -3)) + -4) + -5)) FROM numbers(500000)</query>
<query>SELECT max(-1 + (((-2 + (number + -3)) + -4) + -5)) FROM numbers(500000000)</query>
<query>SELECT max(((((number) * 10) * -2) * 3) * 2) + min(((((number) * 10) * -2) * 3) * 2) FROM numbers(500000000)</query>
<query>SELECT max(-1 + (((-2 + (number + -3)) + -4) + -5)) FROM numbers(500000)</query>
<query>SELECT max(((((number) * 10) * -2) * 3) * 2) + min(((((number) * 10) * -2) * 3) * 2) FROM numbers(500000)</query>
</test>

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT * FROM system.numbers LIMIT 3;
SELECT sys_num.number FROM system.numbers AS sys_num WHERE number > 2 LIMIT 2;

View File

@ -2,7 +2,7 @@ CREATE DATABASE IF NOT EXISTS test2_00158;
DROP TABLE IF EXISTS test2_00158.mt_buffer_00158;
DROP TABLE IF EXISTS test2_00158.mt_00158;
CREATE TABLE test2_00158.mt_buffer_00158 (d Date DEFAULT today(), x UInt64) ENGINE = Buffer(test2_00158, mt_00158, 16, 100, 100, 1000000, 1000000, 1000000000, 1000000000);
SET send_logs_level = 'none'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded."
SET send_logs_level = 'fatal'; -- Supress "Destination table test2.mt doesn't exist. Block of data is discarded."
INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 100000;
INSERT INTO test2_00158.mt_buffer_00158 (x) SELECT number AS x FROM system.numbers LIMIT 1000000;
DROP TABLE IF EXISTS test2_00158.mt_buffer_00158;

View File

@ -1,3 +1,3 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT count() FROM remote('{127,1}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1;
SELECT count() FROM remote('{1,127}.0.0.{2,3}', system.one) SETTINGS skip_unavailable_shards = 1;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT (SELECT (SELECT (SELECT (SELECT (SELECT count() FROM (SELECT * FROM system.numbers LIMIT 10)))))) = (SELECT 10), ((SELECT 1, 'Hello', [1, 2]).3)[1];
SELECT toUInt64((SELECT 9)) IN (SELECT number FROM system.numbers LIMIT 10);

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select 1 = position('', '');
select 1 = position('abc', '');
select 0 = position('', 'abc');

View File

@ -2,7 +2,7 @@ DROP TEMPORARY TABLE IF EXISTS temp_tab;
CREATE TEMPORARY TABLE temp_tab (number UInt64);
INSERT INTO temp_tab SELECT number FROM system.numbers LIMIT 1;
SELECT number FROM temp_tab;
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
EXISTS TEMPORARY TABLE temp_tab;
DROP TABLE temp_tab;
EXISTS TEMPORARY TABLE temp_tab;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS sum_map;
CREATE TABLE sum_map(date Date, timeslot DateTime, statusMap Nested(status UInt16, requests UInt64)) ENGINE = Log;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET any_join_distinct_right_table_keys = 1;
SET joined_subquery_requires_alias = 0;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT countMerge(x) AS y FROM ( SELECT countState() * 2 AS x FROM ( SELECT 1 ));
SELECT countMerge(x) AS y FROM ( SELECT countState() * 0 AS x FROM ( SELECT 1 UNION ALL SELECT 2));

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT if(); -- { serverError 42 }
SELECT if(1); -- { serverError 42 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET max_block_size = 0;
SELECT number FROM system.numbers; -- { serverError 12 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS mergetree_00698;
CREATE TABLE mergetree_00698 (k UInt32, `n.x` Array(UInt64), `n.y` Array(UInt64)) ENGINE = MergeTree ORDER BY k;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS Issue_2231_Invalid_Nested_Columns_Size;
CREATE TABLE Issue_2231_Invalid_Nested_Columns_Size (

View File

@ -1,7 +1,7 @@
DROP TABLE IF EXISTS a1;
DROP TABLE IF EXISTS a2;
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
CREATE TABLE a1(a UInt8, b UInt8) ENGINE=Memory;
CREATE TABLE a2(a UInt8, b UInt8) ENGINE=Memory;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS add_aggregate;
CREATE TABLE add_aggregate(a UInt32, b UInt32) ENGINE = Memory;

View File

@ -37,7 +37,7 @@ ENGINE = MergeTree
PARTITION BY toDate(created_at)
ORDER BY (created_at, id0, id1);
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
ALTER TABLE uuid MODIFY COLUMN id0 UUID; -- { serverError 524 }
ALTER TABLE uuid MODIFY COLUMN id1 UUID; -- { serverError 524 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_ddl = 0;
CREATE DATABASE some_db; -- { serverError 392 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS test_local_1;
DROP TABLE IF EXISTS test_local_2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT formatDateTime(); -- { serverError 42 }
SELECT formatDateTime('not a datetime', 'IGNORED'); -- { serverError 43 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
drop table if exists t1_00729;
create table t1_00729 (id UInt64, val Array(String),nid UInt64, eDate Date)ENGINE = MergeTree(eDate, (id, eDate), 8192);

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT base64Encode(val) FROM (select arrayJoin(['', 'f', 'fo', 'foo', 'foob', 'fooba', 'foobar']) val);
SELECT base64Decode(val) FROM (select arrayJoin(['', 'Zg==', 'Zm8=', 'Zm9v', 'Zm9vYg==', 'Zm9vYmE=', 'Zm9vYmFy']) val);
SELECT base64Decode(base64Encode('foo')) = 'foo', base64Encode(base64Decode('Zm9v')) == 'Zm9v';

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS quorum1;
DROP TABLE IF EXISTS quorum2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT 'value vs value';

View File

@ -1,3 +1,3 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET join_default_strictness = '';
SELECT * FROM system.one INNER JOIN (SELECT number AS k FROM system.numbers) js2 ON dummy = k; -- { serverError 417 }

View File

@ -1,6 +1,6 @@
-- check ALTER MODIFY COLUMN with partitions
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS alter_column;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS old_style;
CREATE TABLE old_style(d Date, x UInt32) ENGINE MergeTree(d, x, 8192);

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS old_style;
CREATE TABLE old_style(d Date, x UInt32) ENGINE ReplicatedMergeTree('/clickhouse/tables/test/old_style', 'r1', d, x, 8192);

View File

@ -11,5 +11,5 @@ SELECT reverse(NULL);
SELECT reverse([]);
SELECT reverse([[[[]]]]);
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT '[RE7', ( SELECT '\0' ) AS riwwq, ( SELECT reverse([( SELECT bitTestAll(NULL) ) , ( SELECT '\0' ) AS ddfweeuy]) ) AS xuvv, '', ( SELECT * FROM file() ) AS wqgdswyc, ( SELECT * FROM file() ); -- { serverError 42 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select today() < 2018-11-14; -- { serverError 43 }
select toDate('2018-01-01') < '2018-11-14';

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select lcase('FOO');
select ucase('foo');

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS alter_compression_codec;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_suspicious_codecs = 1;
DROP TABLE IF EXISTS compression_codec;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_suspicious_codecs = 1;
-- copy-paste for storage log

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET joined_subquery_requires_alias = 0;
DROP TABLE IF EXISTS delta_codec_synthetic;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_suspicious_codecs = 1;
DROP TABLE IF EXISTS delta_codec_for_alter;

View File

@ -4,7 +4,7 @@ SELECT toInt32(number) as x, roundDown(x, [e(), pi(), pi(), e()]) FROM system.nu
SELECT number as x, roundDown(x, [6, 5, 4]) FROM system.numbers LIMIT 10;
SELECT 1 as x, roundDown(x, [6, 5, 4]);
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT 1 as x, roundDown(x, []); -- { serverError 43 }
SELECT 1 as x, roundDown(x, emptyArrayUInt8()); -- { serverError 44 }
SELECT roundDown(number, [number]) FROM system.numbers LIMIT 10; -- { serverError 44 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT arrayEnumerateUniq(anyHeavy([]), []);
SELECT arrayEnumerateDense([], [sequenceCount(NULL)]); -- { serverError 190 }
SELECT arrayEnumerateDense([STDDEV_SAMP(NULL, 910947.571364)], [NULL]);

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS test_00808;
CREATE TABLE test_00808(date Date, id Int8, name String, value Int64, sign Int8) ENGINE = CollapsingMergeTree(sign) ORDER BY (id, date);

View File

@ -4,7 +4,7 @@ SELECT ignore(subtractDays(toDateTime(0), 1));
SELECT ignore(addDays(toDate(0), -1));
SELECT ignore(subtractDays(toDate(0), 1));
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT ignore(addDays((CAST((96.338) AS DateTime)), -3));
SELECT ignore(subtractDays((CAST((-5263074.47) AS DateTime)), -737895));

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT globalNotIn(['"wh'], [NULL]);
SELECT globalIn([''], [NULL]);

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT truncate(895, -16);
SELECT ( SELECT toDecimal128([], rowNumberInBlock()) ) , lcm('', [[(CAST(('>A') AS String))]]); -- { serverError 44 }

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET allow_suspicious_codecs = 1;
DROP TABLE IF EXISTS test.compression_codec_replicated1;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SET replication_alter_partitions_sync = 2;
DROP TABLE IF EXISTS alter_compression_codec1;

View File

@ -42,3 +42,4 @@ SimpleAggregateFunction(sum, Float64)
1 1 2 2.2.2.2 3 ([1,2,3],[2,1,1]) [1,2,2,3,4] [4,2,1,3]
10 2222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222222 20 20.20.20.20 5 ([2,3,4],[2,1,1]) [] []
SimpleAggregateFunction(anyLast, Nullable(String)) SimpleAggregateFunction(anyLast, LowCardinality(Nullable(String))) SimpleAggregateFunction(anyLast, IPv4) SimpleAggregateFunction(groupBitOr, UInt32) SimpleAggregateFunction(sumMap, Tuple(Array(Int32), Array(Int64))) SimpleAggregateFunction(groupArrayArray, Array(Int32)) SimpleAggregateFunction(groupUniqArrayArray, Array(Int32))
with_overflow 1 0

View File

@ -1,3 +1,5 @@
set optimize_throw_if_noop = 1;
-- basic test
drop table if exists simple;
@ -41,3 +43,15 @@ select toTypeName(nullable_str),toTypeName(low_str),toTypeName(ip),toTypeName(st
optimize table simple final;
drop table simple;
create table with_overflow (
id UInt64,
s SimpleAggregateFunction(sumWithOverflow, UInt8)
) engine AggregatingMergeTree order by id;
insert into with_overflow select 1, 1 from numbers(256);
optimize table with_overflow final;
select 'with_overflow', * from with_overflow;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
SELECT '----00489----';
DROP TABLE IF EXISTS pk;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select 0 = multiFuzzyMatchAny('abc', 0, ['a1c']) from system.numbers limit 5;
select 1 = multiFuzzyMatchAny('abc', 1, ['a1c']) from system.numbers limit 5;

View File

@ -1,4 +1,4 @@
set send_logs_level = 'none';
set send_logs_level = 'fatal';
drop table if exists ttl;

View File

@ -83,7 +83,7 @@ select sleep(0.7) format Null; -- wait if very fast merge happen
optimize table ttl_00933_1 final;
select * from ttl_00933_1;
set send_logs_level = 'none';
set send_logs_level = 'fatal';
drop table if exists ttl_00933_1;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
select format('Hello {1} World {0}', materialize('first'), materialize('second')) from system.numbers limit 1;
select format('Hello {0} World {1}', materialize('first'), materialize('second')) from system.numbers limit 2;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP TABLE IF EXISTS mt_with_pk;

View File

@ -26,7 +26,12 @@ done &
sleep 4
# we just test race conditions, not logic
# if we have too many simultaneous queries
until $CLICKHOUSE_CLIENT --query "SELECT 1" 2>/dev/null 1>/dev/null
do
sleep 0.5
done
$CLICKHOUSE_CLIENT --query "SELECT 1"
$CLICKHOUSE_CLIENT --query="DROP TABLE IF EXISTS table_for_concurrent_alter"

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

View File

@ -1,4 +1,4 @@
SET send_logs_level = 'none';
SET send_logs_level = 'fatal';
DROP DATABASE IF EXISTS database_for_dict;

Some files were not shown because too many files have changed in this diff Show More