Commit Graph

72630 Commits

Author SHA1 Message Date
Azat Khuzhin
c77c4e22ae Add MySQL query into after transformation in case of BadQuery
Since it may differs from the user input
2021-09-11 14:06:19 +03:00
Pavel Kruglov
9c756fc4b2 Fix bug with LowCardinality in short-curcuit function evaluation 2021-09-11 12:56:25 +03:00
Maksim Kita
48e8e24552
Merge pull request #28882 from ClickHouse/async-metric-skip-some-exceptions
Skip "no data" exception when reading thermal sensors
2021-09-11 11:36:23 +03:00
Maksim Kita
aef6112af4
Merge pull request #28881 from azat/mysql-sync-thread-joinable
Join MaterializedMySQLSyncThread only if it is joinable
2021-09-11 11:35:43 +03:00
Vladimir C
aee967de3d
Merge pull request #28194 from ka1bi4/romanzhukov-DOCSUP-12543-bin-unbin-functions 2021-09-11 10:47:17 +03:00
Vladimir C
a581603964
Apply suggestions from code review 2021-09-11 10:46:55 +03:00
Artur
c77a65f9d5 avoid error with order 2021-09-11 07:28:54 +00:00
Alexey Milovidov
7e084304fb Skip "no data" exception when reading thermal sensors 2021-09-11 10:03:06 +03:00
Azat Khuzhin
ff17de531c Join MaterializedMySQLSyncThread only if it is joinable
It is possible to trigger
MaterializedMySQLSyncThread::stopSynchronization() from the same thread
in case of some exception at startup, when some interpreter holds the
storage refcnt, and later MaterializedMySQLSyncThread will try to join
itself from the main thread.

Here is a stack trace for example:

<details>

    4  0x000000000f7ae45c in Poco::Event::wait (this=0x7f1b90c38170) at ../contrib/poco/Foundation/include/Poco/Event.h:97
    5  ThreadFromGlobalPool::join (this=0x7f1b90c23120) at ../src/Common/ThreadPool.h:210
    6  DB::MaterializeMySQLSyncThread::stopSynchronization (this=0x7f1b8024ca68) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:229
    7  DB::MaterializeMySQLSyncThread::~MaterializeMySQLSyncThread (this=0x7f1b8024ca68) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:85
    8  0x000000000f816dc8 in DB::DatabaseMaterializeMySQL<DB::DatabaseAtomic>::~DatabaseMaterializeMySQL (this=0x7f1b8024c918) at ../src/Databases/MySQL/DatabaseMaterializeMySQL.h:21
    11 std::__1::shared_ptr<DB::IDatabase>::~shared_ptr (this=<optimized out>) at ../contrib/libcxx/include/memory:3212
    12 0x000000000f8726a6 in DB::InterpreterCreateQuery::createTable (this=<optimized out>, create=...) at ../src/Interpreters/InterpreterCreateQuery.cpp:952
    13 0x000000000f87735c in DB::InterpreterCreateQuery::execute (this=0x7f1aeef59860) at ../src/Interpreters/InterpreterCreateQuery.cpp:1225
    14 0x000000000fe22253 in DB::executeQueryImpl (begin=<optimized out>, end=<optimized out>, context=..., internal=true, stage=DB::QueryProcessingStage::Complete, has_query_tail=<optimized out>, istr=<optimized out>) at ../src/Interpreters/executeQuery.cpp:574
    15 0x000000000fe208e3 in DB::executeQuery (query=..., context=..., internal=<optimized out>, stage=DB::QueryProcessingStage::FetchColumns, may_have_embedded_data=<optimized out>) at ../src/Interpreters/executeQuery.cpp:933
    16 0x000000000faafcde in DB::MySQLInterpreter::InterpreterMySQLDDLQuery<DB::MySQLInterpreter::InterpreterCreateImpl>::execute (this=<optimized out>) at ../src/Interpreters/MySQL/InterpretersMySQLDDLQuery.h:75
    17 0x000000000faade78 in DB::InterpreterExternalDDLQuery::execute (this=<optimized out>) at ../src/Interpreters/InterpreterExternalDDLQuery.cpp:64
    18 0x000000000fe22253 in DB::executeQueryImpl (begin=<optimized out>, end=<optimized out>, context=..., internal=true, stage=DB::QueryProcessingStage::Complete, has_query_tail=<optimized out>, istr=<optimized out>) at ../src/Interpreters/executeQuery.cpp:574
    19 0x000000000fe208e3 in DB::executeQuery (query=..., context=..., internal=<optimized out>, stage=DB::QueryProcessingStage::FetchColumns, may_have_embedded_data=<optimized out>) at ../src/Interpreters/executeQuery.cpp:933
    20 0x000000000f7ba406 in DB::tryToExecuteQuery (query_to_execute=..., query_context=..., database=..., comment=...) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:69
    21 0x000000000f7d4b88 in DB::dumpDataForTables() (connection=..., query_prefix=..., database_name=..., mysql_database_name=..., context=..., is_cancelled=..., need_dumping_tables=...) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:326
    22 DB::MaterializeMySQLSyncThread::prepareSynchronized(DB::MaterializeMetadata&)::$_1::operator()() const (this=<optimized out>) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:391
    29 DB::commitMetadata() (function=..., persistent_tmp_path=..., persistent_path=...) at ../src/Databases/MySQL/MaterializeMetadata.cpp:197
    30 0x000000000f80a000 in DB::MaterializeMetadata::transaction(DB::MySQLReplication::Position const&, std::__1::function<void ()> const&) (this=0x7f1b6375c3d8, position=..., fun=...) at ../src/Databases/MySQL/MaterializeMetadata.cpp:230
    31 0x000000000f7b2231 in DB::MaterializeMySQLSyncThread::prepareSynchronized (this=0x7f1b8024ca68, metadata=...) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:388
    32 0x000000000f7b169c in DB::MaterializeMySQLSyncThread::synchronization (this=0x7f1b8024ca68) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:180
    33 0x000000000f7d4074 in DB::MaterializeMySQLSyncThread::startSynchronization()::$_0::operator()() const (this=<optimized out>) at ../src/Databases/MySQL/MaterializeMySQLSyncThread.cpp:236

</details>
2021-09-11 09:27:12 +03:00
Azat Khuzhin
c6b9816d45 tests: mark 00763_lock_buffer as long
https://clickhouse-test-reports.s3.yandex.net/28827/58266e4ed6db77109e6237ecd2612d34c0b19516/functional_stateless_tests_flaky_check_(address).html#fail1
2021-09-11 09:15:38 +03:00
Azat Khuzhin
f7540dad56 tests: update 01305_replica_create_drop_zookeeper after client query printing 2021-09-11 09:13:23 +03:00
alexey-milovidov
28911a959f
Merge pull request #28431 from azat/thread-pool-joinable
[RFC] Do not allow ThreadFromGlobalPool::join() from the spawned/occupated thread
2021-09-11 05:44:56 +03:00
alexey-milovidov
d50335041d
Merge pull request #28874 from azat/heavy-build-heavy-system
Relax ENABLE_CHECK_HEAVY_BUILDS restrictions on systems under pressure (increase RLIMIT_CPU)
2021-09-11 05:29:32 +03:00
adevyatova
1d4f73ac7b Add example 2021-09-11 02:13:55 +00:00
Mike Kot
bce011cb33 Replacing IsDecimalNumber 2021-09-11 00:40:18 +02:00
Mike Kot
c4385a2c40 Simplifying exception messages 2021-09-11 00:40:18 +02:00
Azat Khuzhin
017dc9fa2a Remove unused ILLEGAL_AGGREGATION from AggregateFunctionIf 2021-09-11 00:48:17 +03:00
Azat Khuzhin
a764d46f62 Eliminate int64_t > INT64_MAX checks (found by PVS)
https://clickhouse-test-reports.s3.yandex.net/28773/b38311538979c0ffe27f6c8c5732b4afc18c4d35/pvs_check.html#fail1
2021-09-10 23:31:41 +03:00
Azat Khuzhin
56ac032413 Use the same error for leadInFrame()/lagInFrame() as in nth_value() 2021-09-10 23:27:36 +03:00
Azat Khuzhin
58266e4ed6 tests/queries/0_stateless/01175_distributed_ddl_output_mode_long: use unique tmp file 2021-09-10 23:23:24 +03:00
Azat Khuzhin
4336038711 tests: update more references after client started printing query 2021-09-10 23:19:16 +03:00
Azat Khuzhin
c362e7c541 Introduce IAggregateFunctionCombinator::supportsNesting() 2021-09-10 22:41:38 +03:00
Azat Khuzhin
d8bf2f81ec Allow Array/OrNull combinators regardless nesting 2021-09-10 22:35:25 +03:00
Azat Khuzhin
4e9b2c5d69 Relax ENABLE_CHECK_HEAVY_BUILDS restrictions on systems under pressure (increase RLIMIT_CPU)
The problem was with compiling src/AggregateFunctions/AggregateFunctionQuantile.cpp
2021-09-10 21:05:41 +03:00
Azat Khuzhin
1e7714d56d Forbid nested identical (only) combinators
Nested identical combinators (i.e. uniqCombinedIfIf) is not
supported (since they even don't work -- silently).

But non-identical does supported and works, for example
uniqCombinedIfMergeIf, it is useful in case when the underlying
storage stores AggregateFunction(uniqCombinedIf) and in SELECT you
need to filter aggregation result based on another column for
example.
2021-09-10 21:03:24 +03:00
Azat Khuzhin
60df1e8800 Relax nested function for If-combinator check
Second If combinator can be useful to add condition based on another
column, when Merge combinator is applied for example (i.e.
uniqCombinedIfMergeIf(state_of_merge_that_is_stored_in_table, event_date = today())).
2021-09-10 21:03:24 +03:00
Azat Khuzhin
c4a24c7696 tests: update reference after client started printing query 2021-09-10 21:02:55 +03:00
Azat Khuzhin
09f43d21a8 client: print query on client exception only in non-interactive mode 2021-09-10 21:02:55 +03:00
Azat Khuzhin
f0f31ab8f3 client: print full query in case of server exception 2021-09-10 21:02:54 +03:00
Azat Khuzhin
af5072db50 client: print full query in case of test hints failures 2021-09-10 21:02:54 +03:00
Azat Khuzhin
b383115389 Allow values up to INT64_MAX for nth_value/lagInFrame/leadInFrame 2021-09-10 21:02:06 +03:00
Azat Khuzhin
62a29420bd Fix signed integer overflow for nth_value() window function
CI report [1]:

    ../src/Processors/Transforms/WindowTransform.cpp:1699:54: runtime error: signed integer overflow: -9223372036854775808 - 1 cannot be represented in type 'long' Received signal -3 Received signal Unknown signal (-3)

In query:

    SELECT
        number,
        nth_value(number, 2147483648) OVER w,
        anyOrNull(number) OVER (ORDER BY -2147483648 - intDiv(number, 1024) DESC NULLS FIRST, number DESC ROWS BETWEEN 65535 FOLLOWING AND UNBOUNDED FOLLOWING),
        nth_value(number, 65537) OVER w AS firstValue,
        nth_value(number, -9223372036854775808) OVER w AS secondValue,
        nth_value(number, 1048576) OVER w AS thirdValue
    FROM numbers(1)
    WINDOW w AS (ORDER BY number DESC)
    ORDER BY number DESC

  [1]: https://clickhouse-test-reports.s3.yandex.net/28532/7623af5513e12aa8dfa1bee963caffe00185c31a/fuzzer_ubsan/report.html#fail1
2021-09-10 21:02:04 +03:00
Anton Popov
8999a83320 add comment 2021-09-10 20:22:28 +03:00
Anton Popov
a63887413c fix reading of subcolumns from compact parts 2021-09-10 20:19:18 +03:00
Filatenkov Artur
acccb24ffd
Update getSourceFromFromASTInsertQuery.cpp 2021-09-10 19:59:17 +03:00
Filatenkov Artur
c92a096d2d
Update Client.cpp 2021-09-10 19:58:52 +03:00
Filatenkov Artur
67df28bee0
Update ParserInsertQuery.cpp 2021-09-10 19:58:15 +03:00
alesapin
21af80baf5
Update docs/ru/operations/settings/merge-tree-settings.md 2021-09-10 18:33:58 +03:00
alesapin
32b61a4bb6
Update docs/ru/operations/settings/merge-tree-settings.md 2021-09-10 18:33:53 +03:00
Filatenkov Artur
4eac0c8021
Update Client.cpp 2021-09-10 18:04:26 +03:00
Filatenkov Artur
e17353750c
Update src/Processors/Transforms/getSourceFromFromASTInsertQuery.cpp
Co-authored-by: Nikita Mikhaylov <mikhaylovnikitka@gmail.com>
2021-09-10 18:02:15 +03:00
Vladimir C
5b967d91ba
Merge pull request #28716 from olgarev/revolg-DOCSUP-13742-partitions_in_s3_table_function 2021-09-10 17:57:58 +03:00
Maksim Kita
a87ffdff92 Column default dictGet identifier fix 2021-09-10 17:41:09 +03:00
Anton Popov
0bb74f8eaf
Merge pull request #28762 from CurtizJ/fix-nested-1
Fix usage of nested columns with non-array columns with the same prefix [2]
2021-09-10 17:34:43 +03:00
Alexander Tokmakov
518f542c3e fix reordering of REPLACE_RANGE and DROP PART 2021-09-10 17:32:45 +03:00
Filatenkov Artur
a117804914
Merge pull request #2 from FArthur-cmd/compress_output_3473
add compression key-word and some tests
2021-09-10 17:04:56 +03:00
Artur
a7c43c3e2d add compression key-word and some tests 2021-09-10 13:59:22 +00:00
alesapin
d1dea0e95f
Update src/Storages/MergeTree/ReplicatedMergeTreePartCheckThread.cpp
Co-authored-by: Azat Khuzhin <a3at.mail@gmail.com>
2021-09-10 15:32:24 +03:00
mergify[bot]
e3b3439535
Merge branch 'master' into fix-build-clang-13-2 2021-09-10 12:05:54 +00:00
Nikita Mikhaylov
56abedfb2b Clang tidy 2021-09-10 12:04:43 +00:00