Commit Graph

7101 Commits

Author SHA1 Message Date
alexey-milovidov
06477c2a7e Update ReplicatedMergeTreeSink.cpp 2022-02-01 09:22:40 +01:00
Igor Nikonov
f4c0b64420 Clean up: insert_deduplication_token setting for INSERT statement
+ reduce number of allocations on replication merge tree path
+ bash test: move insert block settings into variable

Issue: ClickHouse#7461
2022-02-01 09:22:33 +01:00
alexey-milovidov
99392b5ca7
Merge pull request #13544 from amosbird/mdha
Multi-Disk auto-recovery.
2022-02-01 06:13:26 +03:00
Alexey Milovidov
7dbf0dede5 Change severity of the "Cancelled merging parts" message in logs 2022-02-01 05:55:07 +03:00
alexey-milovidov
095d9bfa43
Revert "Add pool to WriteBufferFromS3" 2022-02-01 05:49:40 +03:00
mergify[bot]
e229487817
Merge branch 'master' into mdha 2022-02-01 01:22:16 +00:00
Dmitry Novik
3a0ed0f8f5
Merge pull request #33324 from azat/buffer-different-structure
Simplify different block structure (i.e. after ALTER) support for Buffer
2022-02-01 02:19:49 +03:00
alexey-milovidov
15e4fe5c78
Revert "Additionally check remote_fs_execute_merges_on_single_replica_time_threshold inside ReplicatedMergeTreeQueue" 2022-02-01 01:51:39 +03:00
Amos Bird
ec7d367814
DiskLocal checker
Add DiskLocal checker so that ReplicatedMergeTree can recover data when some of its disks are broken.
2022-02-01 05:55:27 +08:00
alesapin
dd61d1c2de
Merge pull request #34172 from ClickHouse/fix_race_in_some_engines
Fix benign race condition for storage HDFS, S3, URL
2022-01-31 22:41:54 +03:00
alesapin
75d73d2785
Merge pull request #34139 from ClickHouse/fix_buf_s3_low_cardinality
Fix bug with bounded S3 reads and LowCardinality
2022-01-31 22:41:14 +03:00
Nikolai Kochetov
348d72266a
Merge pull request #34189 from ClickHouse/less-logging-for-remote_fs_execute_merges_on_single_replica_time_threshold
Additionally check remote_fs_execute_merges_on_single_replica_time_threshold inside ReplicatedMergeTreeQueue
2022-01-31 21:39:04 +03:00
Nikolai Kochetov
a207cdf28f Additionally check remote_fs_execute_merges_on_single_replica_time_threshold inside ReplicatedMergeTreeQueue. 2022-01-31 17:53:28 +00:00
Yatsishin Ilya
676fc5b20f Initial commit 2022-01-31 17:10:59 +00:00
Nikolai Kochetov
321fa4a9e8
Merge pull request #33291 from ClickHouse/add-pool-to-s3-write-buffer
Add pool to WriteBufferFromS3
2022-01-31 19:37:40 +03:00
Maksim Kita
8513f20cfd
Merge pull request #34145 from kitaisreal/bitset-sort-performance-check
pdqsort performance check
2022-01-31 12:35:13 +01:00
alesapin
5230c59f05 Smaller scope 2022-01-31 14:03:06 +03:00
alesapin
1adc7538fe Fix benign race condition for storage HDFS, S3, URL 2022-01-31 13:50:28 +03:00
tavplubix
d19e24f530
Merge pull request #34096 from ClickHouse/fix_race_merge_selecting_task
Fix race between mergeSelectingTask and queue reinitialization
2022-01-31 12:16:29 +03:00
Kruglov Pavel
a9d0beb7ae
Fix data race in StorageFile (#34113)
* Fix data race in StorageFile

* Update StorageFile.h

* Fix
2022-01-31 11:58:40 +03:00
alesapin
55c7936257 Fix incorrect range for index 2022-01-31 11:11:32 +03:00
Maksim Kita
5ef83deaa6 Update sort to pdqsort 2022-01-30 19:49:48 +00:00
alesapin
4f1b902342 Fix compact parts as well 2022-01-30 22:36:19 +03:00
alesapin
4bedcc19b5 Better invariants 2022-01-30 20:40:09 +03:00
alesapin
c237c03c50 Fix 2022-01-30 18:39:26 +03:00
alesapin
bf918892ac More clear code with less getMark calls 2022-01-30 18:21:05 +03:00
alesapin
cb45a348f1 Merge branch 'master' into fix_buf_s3_low_cardinality 2022-01-30 17:30:55 +03:00
alesapin
3f3e90c8ba Remove redundant code 2022-01-29 23:55:45 +03:00
alexey-milovidov
2b43bad923
Update MergeTreeReaderStream.cpp 2022-01-29 19:28:39 +03:00
alesapin
7ada8227cf Fix bug with bounded S3 reads and LowCardinality 2022-01-29 18:28:40 +03:00
Anton Popov
96a506c6fa fix inserts to distributed tables in case of change of native protocol 2022-01-29 03:23:25 +03:00
alexey-milovidov
6535b75322
Merge pull request #34001 from azat/memory-tracker-fix
Fix memory accounting for queries that uses < max_untracker_memory
2022-01-29 00:59:53 +03:00
Azat Khuzhin
1519985c98 Fix possible "Can't attach query to the thread, it is already attached"
After detachQueryIfNotDetached() had been removed it is not enough to
use attachTo() for ThreadPool (scheduleOrThrowOnError()) since the query
may be already attached, if the thread doing multiple jobs, so
CurrentThread::attachToIfDetached() should be used instead.

This should fix all the places from the failures on CI [1]:

    $ fgrep DB::CurrentThread::attachTo -A1 ~/Downloads/47.txt  | fgrep -v attachTo | cut -d' ' -f5,6 | sort | uniq -c
         92 --
          2 /fasttest-workspace/build/../../ClickHouse/contrib/libcxx/include/deque:1393: DB::ParallelParsingInputFormat::parserThreadFunction(std::__1::shared_ptr<DB::ThreadGroupStatus>,
          4 /fasttest-workspace/build/../../ClickHouse/src/Storages/MergeTree/MergeTreeData.cpp:1595: void
         87 /fasttest-workspace/build/../../ClickHouse/src/Storages/MergeTree/MergeTreeDataSelectExecutor.cpp:993: void

  [1]: https://github.com/ClickHouse/ClickHouse/runs/4954466034?check_suite_focus=true

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-28 16:25:33 +03:00
Azat Khuzhin
b0c862c297 Fix memory accounting for queries that uses < max_untracker_memory
MemoryTracker starts accounting memory directly only after per-thread
allocation exceeded max_untracker_memory (or memory_profiler_step).

But even memory under this limit should be accounted too, and there is
code to do this in ThreadStatus dtor, however due to
PullingAsyncPipelineExecutor detached the query from thread group that
memory was not accounted.

So remove CurrentThread::detachQueryIfNotDetached() from threads that
uses ThreadFromGlobalPool since it has ThreadStatus, and the query will
be detached using CurrentThread::defaultThreadDeleter.

Note, that before this patch memory accounting works for HTTP queries
due to it had been accounted from ParallelFormattingOutputFormat, but
not for TCP.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-28 16:25:33 +03:00
Nikolai Kochetov
1c9f026178 Merge branch 'master' into add-pool-to-s3-write-buffer 2022-01-28 16:01:42 +03:00
Alexander Tokmakov
b3ddc601a5 fix race between mergeSelectingTask and queue reinitialization 2022-01-28 15:50:58 +03:00
liyang830
a69711ccff ExpressionList parse tuple and ttl 2022-01-28 19:07:59 +08:00
alexey-milovidov
f6684dbc62
Merge pull request #32304 from devcrafter/deduplication_token_7461
insert_deduplication_token setting for INSERT statement
2022-01-28 13:03:55 +03:00
Kruglov Pavel
efa8775ea6
Merge pull request #33960 from Avogar/autodetect-format
Detect format and schema from stdin in clickhouse-local
2022-01-28 12:02:15 +03:00
Azat Khuzhin
42ddf4d2b8 Fix negative StorageBufferBytes by proper accounting of empty blocks
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-28 11:10:00 +03:00
Azat Khuzhin
d77790182d Tiny cleanup in StorageBuffer::flushBuffer()
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-28 11:10:00 +03:00
Azat Khuzhin
9948525816 Simplify different block sturcture (i.e. after ALTER) support for Buffer
v2: fix empty block in case of flush

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-28 11:09:56 +03:00
Nikolai Kochetov
94999e86c2
Merge pull request #33392 from azat/_shard_num
Replace old _shard_num implementation with shardNum() function
2022-01-28 11:02:19 +03:00
alexey-milovidov
63d8d75001
Merge pull request #33957 from azat/fix-getauxval
Fix getauxval() in glibc-compatibility and fix some leaks (after LSan started to work)
2022-01-27 23:05:23 +03:00
mergify[bot]
ad146a67ff
Merge branch 'master' into _shard_num 2022-01-27 13:11:42 +00:00
liyang830
c2bdeb349c Merge branch 'fix-substr-zk-metadata' of github.com:sunny19930321/ClickHouse into fix-substr-zk-metadata 2022-01-27 16:35:02 +08:00
liyang830
eca0453564 fix local metadata differ zk metadata 2022-01-27 16:33:40 +08:00
Nikolai Kochetov
9b2998c639 Review fixes. 2022-01-26 18:08:01 +00:00
Nikolai Kochetov
a8171269a1 Review fixes. 2022-01-26 17:55:24 +00:00
Raúl Marín
5a59d976dd CurrentlyExecuting: Require mutex usage explicitly 2022-01-26 18:44:35 +01:00
Azat Khuzhin
08f4f45fd9 Fix Context leak (recursive context reference) in StorageSQLite
CI founds after LSan had been fixed [1]:

    01889_sqlite_read_write:                                                [ FAIL ] 8.32 sec. - return code: 1

    =================================================================
    ==20649==ERROR: LeakSanitizer: detected memory leaks

    Indirect leak of 1968 byte(s) in 1 object(s) allocated from:
        0 0xc5c1ffd in operator new(unsigned long) (/usr/bin/clickhouse+0xc5c1ffd)
        1 0x25e32d0d in std::__1::__unique_if<DB::StorageInMemoryMetadata>::__unique_single std::__1::make_unique<DB::StorageInMemoryMetadata, DB::StorageInMemoryMetadata const&>(DB::StorageInMemoryMetadata c>
        2 0x25e32d0d in DB::IStorage::setInMemoryMetadata(DB::StorageInMemoryMetadata const&) obj-x86_64-linux-gnu/../src/Storages/IStorage.h:194:22
        3 0x29bdee98 in DB::StorageSQLite::StorageSQLite(DB::StorageID const&, std::__1::shared_ptr<sqlite3>, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char> > const&, std:>
        4 0x25ee61d6 in std::__1::shared_ptr<DB::StorageSQLite> shared_ptr_helper<DB::StorageSQLite>::create<DB::StorageID, std::__1::shared_ptr<sqlite3> const&, std::__1::basic_string<char, std::__1::char_tr>
        5 0x25ee61d6 in DB::TableFunctionSQLite::executeImpl(std::__1::shared_ptr<DB::IAST> const&, std::__1::shared_ptr<DB::Context const>, std::__1::basic_string<char, std::__1::char_traits<char>, std::__1:>

    SUMMARY: AddressSanitizer: 171256 byte(s) leaked in 130 allocation(s).

  [1]: https://github.com/ClickHouse/ClickHouse/runs/4929706698?check_suite_focus=true

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-26 12:56:27 +03:00
Nikolai Kochetov
fcc29dbd15 Try to fix integration tests. 2022-01-25 15:26:36 +00:00
Kruglov Pavel
7873b4475f
Merge branch 'master' into autodetect-format 2022-01-25 10:56:52 +03:00
Kruglov Pavel
9f12f4af13
Merge pull request #33302 from Avogar/formats-with-suffixes
Allow to create new files on insert for File/S3/HDFS engines
2022-01-25 10:56:15 +03:00
avogar
a6740d2f9a Detect format and schema for stdin in clickhouse-local 2022-01-25 10:25:37 +03:00
Nikolai Kochetov
52b65c3454 Fix style 2022-01-24 16:21:27 +00:00
Nikolai Kochetov
506ee8c024 Refactor some code. 2022-01-24 15:55:29 +00:00
Nikolai Kochetov
4afa75e80b Fix deduplication. 2022-01-24 14:43:36 +00:00
Nikolai Kochetov
2882e46654 Remove debug logging. 2022-01-24 11:48:00 +00:00
tavplubix
da9a38655b
Merge pull request #33847 from ClickHouse/fix_33806
Fix logical error on RESTORE REPLICA
2022-01-24 13:28:37 +03:00
tavplubix
766786b570
Update StorageReplicatedMergeTree.h 2022-01-24 12:16:13 +03:00
tavplubix
34baca1298
Update src/Storages/StorageReplicatedMergeTree.cpp
Co-authored-by: Bharat Nallan <bharatnc@gmail.com>
2022-01-24 12:12:08 +03:00
tavplubix
2f566d2af9
Update src/Storages/StorageReplicatedMergeTree.cpp
Co-authored-by: Bharat Nallan <bharatnc@gmail.com>
2022-01-24 12:12:03 +03:00
Nikolai Kochetov
f74cf1e152 Merge branch 'master' into add-pool-to-s3-write-buffer 2022-01-21 21:55:42 +03:00
Nikolai Kochetov
a024ddd1d4 Merge branch 'add-pool-to-s3-write-buffer' of github.com:ClickHouse/ClickHouse into add-pool-to-s3-write-buffer 2022-01-21 18:42:24 +00:00
Nikolai Kochetov
b3cbb63487 Merge branch 'master' into add-pool-to-s3-write-buffer 2022-01-21 21:41:54 +03:00
alexey-milovidov
43f2e2c178
Merge pull request #33849 from CurtizJ/fix-sparse-columns
Fix sparse columns
2022-01-21 21:13:02 +03:00
Nikolai Kochetov
ae1b669fca Even more async writes. 2022-01-21 15:55:19 +00:00
Kruglov Pavel
daba57a933
Merge pull request #33737 from Avogar/bool-type
Some improvements and fixes for Bool data type
2022-01-21 15:24:58 +03:00
Alexander Tokmakov
8921725d71 fix logical error on RESTORE REPLICA 2022-01-21 14:32:19 +03:00
taiyang-li
22dc786dd8 optimize log level 2022-01-21 12:16:49 +08:00
Anton Popov
c2206c0b96 better storing serialization infos 2022-01-21 03:29:30 +03:00
Anton Popov
f78fb247a7 fix column ttl 2022-01-21 02:22:16 +03:00
Anton Popov
6c4c9bc04a
Merge pull request #33234 from CurtizJ/fix-flush-in-memory-parts
Fix flushing of in-memory parts
2022-01-21 02:18:03 +03:00
Anton Popov
558eb7ea73 fix sparse columns 2022-01-21 00:06:32 +03:00
Maksim Kita
56a54af27b
Merge pull request #33833 from amosbird/keyconditionfix
Fix KeyCondition with no common types
2022-01-20 20:31:27 +01:00
李扬
9f6f84259d
Fix bug of check table when creating data part with wide format and projection (#33774) 2022-01-20 18:05:17 +03:00
Amos Bird
e62f4e50a6
Fix KeyCondition with no common types 2022-01-20 21:56:12 +08:00
Kruglov Pavel
7bfb1231b9
Merge branch 'master' into formats-with-suffixes 2022-01-20 14:47:17 +03:00
avogar
ba54a5ff80 Some improvements and fixes for Bool data type 2022-01-20 14:16:11 +03:00
alexey-milovidov
28a9d569cc
Merge pull request #33690 from azat/remove-unbundled-support
Remove unbundled support
2022-01-20 12:29:07 +03:00
Kseniia Sumarokova
064c3dd654
Merge pull request #33724 from liuneng1994/fix_ignore_hdfs_error
Throw exception when storage hdfs list directory failed
2022-01-20 11:59:09 +03:00
Azat Khuzhin
b23053d1f6 Remove USE_XXHASH (reduntant) 2022-01-20 10:02:02 +03:00
Azat Khuzhin
e0e81b340d Fix w/o ODBC build 2022-01-20 10:02:02 +03:00
Azat Khuzhin
9426e6305b Fix USE_XXHASH in StorageSystemBuildOptions.generated.cpp.in 2022-01-20 10:02:02 +03:00
Azat Khuzhin
e160a20769 Remove USE_OPENCL (removed in #31744) 2022-01-20 10:02:02 +03:00
Azat Khuzhin
6dcb09f9e1 Cleanup llvm 2022-01-20 10:02:01 +03:00
Azat Khuzhin
614f86edbb Remove unbundled gsasl support 2022-01-20 10:02:00 +03:00
Azat Khuzhin
a773e7ff01 Remove unbundled libpqxx support 2022-01-20 10:01:59 +03:00
Azat Khuzhin
789dfd9f3b Remove unbundled re2 support
v2: preserve re2_st name to make PVS check pass (since docker image
update fails)
2022-01-20 10:00:49 +03:00
liuneng1994
59f3fe515b change null check position
Signed-off-by: neng.liu <neng.liu@kyligence.io>
2022-01-20 06:26:34 +00:00
Kseniia Sumarokova
19df6413dc
Merge pull request #33729 from Vxider/windowview-stream-table-join
Enable stream to table join in WindowView
2022-01-20 09:14:53 +03:00
Azat Khuzhin
aef6668cb4 Remove unbundled hdfs3 support 2022-01-20 08:47:16 +03:00
Azat Khuzhin
8485abd52b Remove unbundled lz4 support 2022-01-20 08:47:16 +03:00
alexey-milovidov
994584df09
Merge pull request #33753 from azat/mt-fix-count-race
Fix tiny race between count() and INSERT/merges/... in MergeTree
2022-01-20 06:33:54 +03:00
mergify[bot]
cbb87e762c
Merge branch 'master' into formats-with-suffixes 2022-01-19 17:21:18 +00:00
Amos Bird
62441f0a0f
Fix mutation when table contains projections (#33679) 2022-01-19 15:27:11 +03:00
tavplubix
cdcd93330a
Merge pull request #33743 from ClickHouse/better_exception_message
Better exception text on suspicious broken parts
2022-01-19 11:25:59 +03:00
mergify[bot]
bfbcb69e20
Merge branch 'master' into windowview-stream-table-join 2022-01-19 07:46:00 +00:00
liuneng1994
4d3bb1584d fix Dereference of null pointer error
Signed-off-by: neng.liu <neng.liu@kyligence.io>
2022-01-19 07:41:46 +00:00
Nikolay Degterinsky
eaad8eef7f
Merge pull request #33731 from Vxider/remove_unused_var
Remove unused variable
2022-01-19 10:35:18 +03:00
Vxider
13cee6c184 update code style 2022-01-19 02:24:27 +00:00
mergify[bot]
f2da296990
Merge branch 'master' into windowview-stream-table-join 2022-01-19 02:08:52 +00:00
alexey-milovidov
d222cb9771
Merge pull request #33695 from amosbird/buildimprove
Some build improvement
2022-01-19 03:19:22 +03:00
Azat Khuzhin
c9e93b21c2 Fix tiny race between count() and INSERT/merges/... in MergeTree
Before it was possible to return stale counter from
StorageMergeTree::totalRows() (that is used for
optimize_trivial_count_query) since the accounting is done in two steps:
- subtract old number of rows
  <-- here the number can be zero, even though there are rows -->
- add new number of rows

This was found by CI [1] in 01615_random_one_shard_insertion test:

Here you can see that INSERT went to both tables:

<details>

    2022.01.16 09:07:34.288252 [ 154369 ] {a1905be0-93da-460c-8c6f-9b5adace72a0} <Debug> DistributedBlockOutputStream: It took 0.035197041 sec. to insert 100
    blocks, 2841.1479249065287 rows per second. Insertion status:
    Wrote 54 blocks and 54 rows on shard 0 replica 0, localhost:9000 (average 0 ms per block, the slowest block 1 ms)
    Wrote 46 blocks and 46 rows on shard 1 replica 0, localhost:9000 (average 0 ms per block, the slowest block 1 ms)

</details>

But the test fails, since select from shard1.tbl returns 0, and the
problem was concurrent merge:

<details>

    2022.01.16 09:07:34.289470 [ 146495 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Debug> executeQuery: (from [::1]:42082) (comment: 01615_random_one_shard_insertion.sql) select count() != 0 from shard_0.tbl;
    2022.01.16 09:07:34.289564 [ 375 ] {c7a885fa-4ef4-4dcf-a4de-1650d44fa0ab::all_1_54_9} <Debug> MergeTask::MergeProjectionsStage: Merge sorted 54 rows, containing 1 columns (1 merged, 0 gathered) in 0.00171193 sec., 31543.345814373253 rows/sec., 246.43 KiB>
    2022.01.16 09:07:34.289810 [ 375 ] {c7a885fa-4ef4-4dcf-a4de-1650d44fa0ab::all_1_54_9} <Trace> shard_0.tbl (c7a885fa-4ef4-4dcf-a4de-1650d44fa0ab): Renaming temporary part tmp_merge_all_1_54_9 to all_1_54_9.
    2022.01.16 09:07:34.289858 [ 146495 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> ContextAccess (default): Access granted: SELECT(number) ON shard_0.tbl
    2022.01.16 09:07:34.289897 [ 375 ] {c7a885fa-4ef4-4dcf-a4de-1650d44fa0ab::all_1_54_9} <Trace> shard_0.tbl (c7a885fa-4ef4-4dcf-a4de-1650d44fa0ab) (MergerMutator): Merged 6 parts: from all_1_49_8 to all_54_54_0
    2022.01.16 09:07:34.289920 [ 146495 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> InterpreterSelectQuery: WithMergeableState -> Complete
    2022.01.16 09:07:34.289987 [ 375 ] {} <Debug> MemoryTracker: Peak memory usage Mutate/Merge: 3.12 MiB.
    2022.01.16 09:07:34.290305 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> MergingAggregatedTransform: Reading blocks of partially aggregated data.
    2022.01.16 09:07:34.290332 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Debug> MergingAggregatedTransform: Read 1 blocks of partially aggregated data, total 1 rows.
    2022.01.16 09:07:34.290343 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> Aggregator: Merging partially aggregated single-level data.
    2022.01.16 09:07:34.290358 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> Aggregator: Merged partially aggregated single-level data.
    2022.01.16 09:07:34.290366 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Trace> Aggregator: Converting aggregated data to blocks
    2022.01.16 09:07:34.290391 [ 154344 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Debug> Aggregator: Converted aggregated data to blocks. 1 rows, 8.00 B in 1.0939e-05 sec. (91416.034 rows/sec., 714.19 KiB/sec.)
    2022.01.16 09:07:34.290709 [ 146495 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Information> executeQuery: Read 1 rows, 4.01 KiB in 0.001187722 sec., 841 rows/sec., 3.30 MiB/sec.
    2022.01.16 09:07:34.290774 [ 146495 ] {cd9d4cf2-7131-4179-b0b2-3aeec4045755} <Debug> MemoryTracker: Peak memory usage (for query): 0.00 B.

</details>

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/33675/7848ea7d609e4c720e8e4494eb6207c0751f5aea/stateless_tests__ubsan__actions_.html

This also fixes a race between DROP TABLE check and INSERT/merges.

v0: use Active parts instead.
v2: fix total counters accounting instead.
2022-01-18 23:58:02 +03:00
avogar
4efadfad3c Fix tests 2022-01-18 22:26:13 +03:00
alesapin
127365658d Followup 2022-01-18 22:00:52 +03:00
alesapin
e7ee116358
Update src/Storages/MergeTree/MergeTreeData.cpp
Co-authored-by: tavplubix <avtokmakov@yandex-team.ru>
2022-01-18 22:00:23 +03:00
alesapin
24f4a5040d Better exception message 2022-01-18 21:59:54 +03:00
alesapin
358e3a165c Better exception text on suspicious broken parts 2022-01-18 19:28:58 +03:00
alesapin
e5ec1aa446
Merge pull request #33642 from ClickHouse/add_integration_test_for_zero_duplication
Fix bug with background moves in zero copy replication
2022-01-18 18:32:16 +03:00
liuneng1994
5e1e512bf2 fix check style
Signed-off-by: neng.liu <neng.liu@kyligence.io>
2022-01-18 14:42:28 +00:00
Kruglov Pavel
2295a07066
Merge pull request #33534 from azat/fwd-decl
RFC: Split headers, move SystemLog into module, more forward declarations
2022-01-18 17:22:49 +03:00
liuneng1994
15ea37c4ae ignore file not found exception
Signed-off-by: neng.liu <neng.liu@kyligence.io>
2022-01-18 12:49:40 +00:00
Vxider
fb62960836 remove unused variable 2022-01-18 09:34:26 +00:00
Vxider
ba0a702807 enable stream to table join in windowview 2022-01-18 09:20:15 +00:00
Vxider
f526ff2a51 remove unused variable 2022-01-18 09:03:16 +00:00
alesapin
eff85d3352 Review fixes 2022-01-18 11:27:01 +03:00
mergify[bot]
df869c5b9c
Merge branch 'master' into fix_ignore_hdfs_error 2022-01-18 07:05:24 +00:00
liuneng1994
c6e205b2ad add error check 2022-01-18 06:21:40 +00:00
Amos Bird
6d62060e16
Build improvement 2022-01-17 22:36:27 +08:00
alesapin
31fb93330e Fix comment 2022-01-17 14:54:51 +03:00
alesapin
4caef03e6a Add ephemeral lock in zookeeper 2022-01-17 14:52:51 +03:00
alesapin
ccf990e887 Merge branch 'master' into add_integration_test_for_zero_duplication 2022-01-17 13:33:39 +03:00
Nikolai Kochetov
6d49a62666 Some more async writes. 2022-01-14 19:53:55 +00:00
avogar
af54ca4866 Fix partition write 2022-01-14 21:21:36 +03:00
Kruglov Pavel
a7df9cd53a
Merge branch 'master' into formats-with-suffixes 2022-01-14 21:03:49 +03:00
avogar
253035a5df Fix 2022-01-14 19:17:06 +03:00
alesapin
bf803472b1 Add a tests for a TTL bug in zero copy replication 2022-01-14 18:44:10 +03:00
Kruglov Pavel
d2e9f37bee
Merge branch 'master' into format-by-extention 2022-01-14 18:36:23 +03:00
avogar
89a181bd19 Make better 2022-01-14 18:16:18 +03:00
Kruglov Pavel
5a908e8edd
Merge branch 'master' into formats-with-suffixes 2022-01-14 16:45:20 +03:00
avogar
8884c4a3fe Fix tests and build 2022-01-14 16:27:57 +03:00
Azat Khuzhin
c341b3b237 Add current database to table names in JOIN section for distributed queries
This should fix JOIN w/o explicit database.

v2: rewrite only JOIN section, since there is old behavior that relies
on default_database for IN section, see [1]:
- 01487_distributed_in_not_default_db
- 01152_cross_replication

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/33611/d0ea3c76fa51131171b1825939680867eb1c04da/fast_test__actions_.html

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-14 11:23:38 +03:00
Azat Khuzhin
0a9b1ee803 Remove RestoreQualifiedNamesMatcher::Data::rename (always true)
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2022-01-14 11:18:52 +03:00
avogar
2d7b1bfa5e Detect format in S3/HDFS/URL table engines 2022-01-13 16:14:18 +03:00
taiyang-li
82a3024712 Merge remote-tracking branch 'ck/master' into bigo_hive_table 2022-01-13 18:11:23 +08:00
alexey-milovidov
6c71a7c40f
Merge pull request #33518 from hexiaoting/dev-limit-offset
Fix bug for select from VIEW with format and limit settings
2022-01-12 23:39:31 +03:00
Nikolai Kochetov
843983ea06 Add batch removing to IDiskl. 2022-01-12 15:58:04 +00:00
avogar
8390e9ad60 Detect format by file name in file/hdfs/s3/url table functions 2022-01-12 18:29:31 +03:00
taiyang-li
66813a3aa9 merge master 2022-01-12 16:56:29 +08:00
Nikolai Kochetov
28f2012b06 Add some more async writing. 2022-01-11 19:02:48 +00:00
hexiaoting
5c847264f7 Fix bug for select from VIEW with format and limit settings 2022-01-11 20:19:41 +08:00
Kseniia Sumarokova
6587cd0ec6
Merge pull request #33231 from kssenii/settings-changes-with-named-conf
Pass settings as key value or config for storage with settings
2022-01-11 12:30:41 +03:00
Kruglov Pavel
94cb98df28
Merge pull request #33177 from kssenii/materialized-postgresql-better-startup
materialized postgresql better startup
2022-01-11 12:18:29 +03:00
Maksim Kita
8465de9a4e
Merge pull request #33482 from zhongyuankai/materialized_view_support_virtual_column
Materialized view support virtual column
2022-01-10 23:13:31 +03:00
Mikhail f. Shiryaev
1cf46f378f
Merge pull request #33494 from msaf1980/rollup_rules_type
GraphiteMergeTree Rollup rules type
2022-01-10 20:53:20 +01:00
Azat Khuzhin
cb70544dfe Move LockMemoryExceptionInThread and MemoryTrackerBlockerInThread 2022-01-10 22:39:10 +03:00
Azat Khuzhin
aee034a597 Use explicit template instantiation for SystemLog
- Move some code into module part to avoid dependency from IStorage in SystemLog
- Remove extra headers from SystemLog.h
- Rewrite some code that was relying on headers that was included by SystemLog.h

v2: rebase
v3: squash move into module part with explicit template instantiation
    (to make each commit self compilable after rebase)
2022-01-10 22:01:41 +03:00
Azat Khuzhin
1637c41d42 Remove leftovers of old _shard_num via identifier implementation 2022-01-10 21:21:24 +03:00
Azat Khuzhin
6671e86c89 Export IStorage::isVirtualColumn() 2022-01-10 21:21:24 +03:00
mergify[bot]
5857667453
Merge branch 'master' into materialized_view_support_virtual_column 2022-01-10 16:25:46 +00:00
Anton Popov
46b9279d81
Merge pull request #33300 from CurtizJ/support-prewhere-storage-merge
Support `optimize_move_to_prewhere` with storage `Merge`
2022-01-10 15:15:15 +03:00
msaf1980
780a1b2abe graphite: split tagged/plain rollup rules (for merges perfomance) 2022-01-10 16:34:16 +05:00
kssenii
21c34ad59b Add support for dictionary source 2022-01-10 14:00:03 +03:00
zhongyuankai
61ea0837d4 materialized view support virtual column 2022-01-10 15:53:41 +08:00
Kseniia Sumarokova
0a1300d32a
Merge pull request #33340 from Vxider/fix-windowview-inner-table-parser
Fix inner table parser in window view
2022-01-10 09:45:17 +03:00
Vxider
ab593d24f3 update code style 2022-01-10 02:16:07 +00:00
kssenii
1984a80750 Merge remote-tracking branch 'upstream/master' into materialized-postgresql-better-startup 2022-01-09 21:23:49 +03:00
Igor Nikonov
102b9cdcfb Merge remote-tracking branch 'origin/master' into deduplication_token_7461 2022-01-09 19:15:33 +01:00
Kseniia Sumarokova
d0a847befc
Merge pull request #33200 from kssenii/fix-materialized-postgresql-ddl-validation
materialized postgresql fix ddl validation
2022-01-09 21:12:13 +03:00
Igor Nikonov
0857a8d061 insert_deduplication_token support in non-replicated MergeTree 2022-01-09 19:06:09 +01:00
kssenii
bfaab1356f Merge remote-tracking branch 'upstream/master' into settings-changes-with-named-conf 2022-01-09 17:56:44 +03:00
kssenii
8f9920dd4d One more fix 2022-01-09 17:46:33 +03:00
kssenii
6639a93735 Better 2022-01-09 12:16:12 +03:00
kssenii
61f85e9afc More fixes 2022-01-08 20:44:17 +03:00
Kseniia Sumarokova
9b3493ba9a
Merge pull request #33468 from kssenii/fix-postgres-db-drop
Fix drop materialized postgres db
2022-01-08 19:41:24 +03:00
alesapin
733ed7cb54
Merge pull request #32061 from ianton-ru/MDB-15474
Change ZooKeeper path for zero-copy locks for shared data
2022-01-08 17:17:11 +03:00
kssenii
c94f4163f7 Merge remote-tracking branch 'upstream/master' into fix-materialized-postgresql-ddl-validation 2022-01-08 15:56:46 +03:00
kssenii
adaeb96b6d Fix 2022-01-08 15:46:22 +03:00
kssenii
64538cf20f Merge remote-tracking branch 'upstream/master' into materialized-postgresql-better-startup 2022-01-08 00:49:08 +03:00
kssenii
90972e3752 Fix race 2022-01-08 00:27:29 +03:00
Azat Khuzhin
28e64020bd Fix query cancellation in case of allow_experimental_parallel_reading_from_replicas
CI found one hanged query [1], where the problem was that becase of
allow_experimental_parallel_reading_from_replicas the Cancel packet was
read by receivePartitionMergeTreeReadTaskResponseAssumeLocked() and so
the executor was not cancelled, while the code in
MergeTreeBaseSelectProcessor was not ready for this:

<details>

    {
        "is_initial_query": 0,
        "elapsed": 1727.714379573,
        "is_cancelled": 0,
        "read_rows": "196577",
        "read_bytes": "1179462",
        "written_rows": "0",
        "written_bytes": "0",
        "query": "SELECT `CounterID`, `EventDate` FROM `test`.`hits` ORDER BY `CounterID` DESC, `EventDate` ASC LIMIT 50",
        ...
     }

In logs:

    2021.12.31 12:11:55.384735 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> executeQuery: (from [::ffff:127.0.0.1]:58094, initial_query_id: e2966ca5-e836-44ef-8f8e-d1c1b32a>
    2021.12.31 12:11:55.454379 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Trace> ContextAccess (default): Access granted: SELECT(EventDate, CounterID) ON test.hits
    2021.12.31 12:11:55.457583 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Trace> datasets.hits_v1: Parallel reading from replicas enabled true
    2021.12.31 12:11:55.459739 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Trace> InterpreterSelectQuery: FetchColumns -> WithMergeableStateAfterAggregationAndLimit
    2021.12.31 12:11:55.471048 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> datasets.hits_v1 (SelectExecutor): Key condition: unknown
    2021.12.31 12:11:55.476514 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> datasets.hits_v1 (SelectExecutor): MinMax index condition: unknown
    2021.12.31 12:11:55.488302 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> datasets.hits_v1 (SelectExecutor): Selected 2/2 parts by partition key, 2 parts by primary key, >
    2021.12.31 12:11:55.494020 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> MergeTreeReverseSelectProcessor: Reading 1 ranges in reverse order from part 201403_20_20_0, app>
    2021.12.31 12:11:55.497644 [ 101532 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> MergeTreeReverseSelectProcessor: Reading 138 ranges in reverse order from part 201403_19_19_2, a>
    2021.12.31 12:11:55.536372 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Compact part, average mark size is 83886080
    2021.12.31 12:11:55.558783 [ 171701 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.563960 [ 171701 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.577512 [ 171701 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.585660 [ 171701 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.613694 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.730597 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.743554 [ 171701 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Debug> MergingSortedTransform: Merge sorted 3 blocks, 65567 rows in 0.243999671 sec., 268717.5754429603>
    2021.12.31 12:11:55.744196 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.890923 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069
    2021.12.31 12:11:55.891222 [ 170601 ] {7e5f551e-5960-4fda-9447-9bfdae4660c1} <Test> MergeTreeBaseSelectProcessor: Reading from Wide part, average mark size is 3069

</details>

  [1]: https://s3.amazonaws.com/clickhouse-test-reports/33341/0685fd99855bacd0bce02507c00a3bd7709eea61/stress_test__address__actions_.html
2022-01-07 20:04:57 +03:00
mergify[bot]
6fcd5a7d41
Merge branch 'master' into MDB-15474 2022-01-07 10:12:25 +00:00
alesapin
b1d2bdf569
Merge pull request #33130 from kssenii/validate-config
Validate config keys for external dictionaries
2022-01-07 12:59:02 +03:00
Maksim Kita
4c1f61bd19
Merge pull request #33431 from bigo-sg/add_build_options
Add some building options in system.build_options
2022-01-07 00:02:57 +03:00
feng lv
90a61b84e5 fix misleading log message 2022-01-06 12:48:38 +00:00
taiyang-li
ede65e3b6e add some building options 2022-01-06 16:12:59 +08:00
lgbo-ustc
f78564491c code style 2022-01-05 14:40:02 +08:00
lgbo-ustc
624da11ee2 fix a bug : cache files are released in wrong way 2022-01-05 14:29:34 +08:00
Alexey Milovidov
ff779b6409 Fix error in #33124 2022-01-05 06:33:28 +03:00
lgbo-ustc
717871d9ce code style
1. remove unused error codes
2022-01-04 17:21:29 +08:00
李扬
2df2442ad0
Merge branch 'master' into hive_table 2022-01-04 01:26:16 -06:00
lgbo-ustc
45c86757bf change LRUCache to LRUResourceCache 2022-01-04 15:19:09 +08:00
taiyang-li
266ca90575 fix fast test 2022-01-04 15:17:43 +08:00
taiyang-li
8e675e9e0a fix building 2022-01-04 15:06:19 +08:00
taiyang-li
05e09e6061 fix typo 2022-01-04 14:30:01 +08:00
taiyang-li
1e102bc1b2 merge master 2022-01-01 09:01:06 +08:00
alexey-milovidov
3f1810c049
Merge pull request #33327 from ClickHouse/rename_committed_to_active
Rename Committed data part state to Active
2021-12-31 14:40:35 +03:00
Vxider
3761994f68 fix inner table parser in window view 2021-12-31 07:07:01 +00:00
Alexander Tokmakov
187c43eba8 rename Committed state to Active 2021-12-30 23:45:38 +03:00
Vladimir C
b46a1a3741
Merge pull request #33312 from zhongyuankai/optimize_system_tables 2021-12-30 17:40:39 +03:00
Vladimir C
794e906ffd
Merge pull request #33218 from amosbird/betternullablekey 2021-12-30 17:13:44 +03:00
vdimir
2686a189de
Apply suggestion for code review, changes in StorageSystemTables getFilteredTables
Co-authored-by: Azat Khuzhin <a3at.mail@gmail.com>
2021-12-30 15:36:38 +03:00
kssenii
bfc705c098 Better 2021-12-30 15:19:17 +03:00
alesapin
91e1ac437e Tiny improvements 2021-12-30 12:57:38 +03:00
alesapin
cbdba89d65 Merge branch 'master' into ianton-ru-MDB-15474 2021-12-30 11:21:19 +03:00
OnePiece
0c714e8c7e
Update StorageSystemTables.cpp 2021-12-30 13:37:27 +08:00
OnePiece
b14f485f85
Update StorageSystemTables.cpp 2021-12-30 13:13:41 +08:00
OnePiece
f3f0ff0f23
Update StorageSystemTables.cpp 2021-12-30 10:12:37 +08:00
alesapin
8b331cd9bb Remove method from IStorage 2021-12-29 22:41:47 +03:00
avogar
131d49e6be Remove comments 2021-12-29 21:26:23 +03:00
avogar
97788b9c21 Allow to create new files on insert for File/S3/HDFS engines 2021-12-29 21:19:13 +03:00
Anton Popov
abcc48c1f6 Merge remote-tracking branch 'upstream/master' into support-prewhere-storage-merge 2021-12-29 21:07:35 +03:00
Anton Popov
7c6f7f6732 support 'optimize_move_to_prewhere' with storage 'Merge' 2021-12-29 20:49:10 +03:00
alesapin
7a3c874651 Merge branch 'master' into ianton-ru-MDB-15474 2021-12-29 19:01:43 +03:00
Nikolai Kochetov
68bf38133a Add pool to WriteBufferFromS3 2021-12-29 14:04:21 +00:00
Kruglov Pavel
dd7f61b337
Merge branch 'master' into schema-inference 2021-12-29 12:59:05 +03:00
Maksim Kita
a5256e5b6e
Merge pull request #30957 from kitaisreal/executable-refactoring
Executable refactoring
2021-12-29 12:43:41 +03:00
avogar
8436638a89 Fix style 2021-12-29 12:21:01 +03:00
avogar
26abf7aa62 Remove code duplication, use simdjson and rapidjson instead of Poco 2021-12-29 12:21:01 +03:00
avogar
aaf9f85c67 Add more tests and fixes 2021-12-29 12:18:56 +03:00
avogar
8112a71233 Implement schema inference for most input formats 2021-12-29 12:18:56 +03:00
Anton Popov
4ebf61b809 proper checking of atomic flags 2021-12-29 01:03:55 +03:00
Maksim Kita
df95a40bbf Fixed tests 2021-12-28 23:50:48 +03:00
Maksim Kita
9ef359ce2c Dictionaries fix comment 2021-12-28 23:50:48 +03:00
Maksim Kita
d1db3c9f42 Updated ShellCommandSource 2021-12-28 22:55:31 +03:00
Maksim Kita
3386378050 Updated test_executable_table_function tests 2021-12-28 22:55:31 +03:00
Maksim Kita
c2977dbf58 Updated test_executable_table_function integration tests 2021-12-28 22:55:30 +03:00
Maksim Kita
5590cfa3aa Updated executable function integration tests 2021-12-28 22:55:30 +03:00
Maksim Kita
2ffd83ee83 Fix shell command source start 2021-12-28 22:55:30 +03:00
Maksim Kita
eb4e400c54 Executable refactoring 2021-12-28 22:55:30 +03:00
Kseniia Sumarokova
0c41b46e75
Merge pull request #33187 from kssenii/materialized-postgresql-fix-cleanup
materialized postgresql make sure temporary replication slots are deleted
2021-12-28 22:04:32 +03:00
alexey-milovidov
934c1b3e83
Merge pull request #33238 from kssenii/some-killing
Allow to cancel some sources which failed to cancel
2021-12-28 22:00:44 +03:00
kssenii
e9b4439a47 Fix 2021-12-28 15:08:40 +03:00
alesapin
ccbc0a8ea5
Merge pull request #33225 from zhongyuankai/optimize_merge_tree_parts_mover
Optimize MergeTreePartsMover
2021-12-28 14:35:43 +03:00
taiyang-li
2f0982d380 fix style 2021-12-28 19:04:23 +08:00
taiyang-li
aa97634d9b merge master 2021-12-28 18:38:50 +08:00
lgbo-ustc
26fcf1fbc7 fixed a bug in allocating local file cache 2021-12-28 16:57:07 +08:00
Anton Ivashkin
2d87f0a0e3 Fix debug build 2021-12-28 11:45:38 +03:00
kssenii
61a2516c8c Merge remote-tracking branch 'upstream/master' into validate-config 2021-12-28 11:30:30 +03:00
Vladimir C
bb6fc853e3
Merge pull request #31442 from vdimir/cluster-discovery 2021-12-28 10:53:30 +03:00
lgbo-ustc
1ac697cad2 some modifications 2021-12-28 11:26:39 +08:00
kssenii
1f6ca619b7 Allow some killing 2021-12-27 22:42:56 +03:00
Anton Ivashkin
0465aef15d Fixes by code review responces 2021-12-27 19:27:06 +03:00
Anton Popov
a83832c03e fix flushing of in-memory parts 2021-12-27 18:54:28 +03:00
Kseniia Sumarokova
0178fb5e0b
Merge pull request #33214 from kssenii/external-database-fix-where-false
Fix WHERE 1=0 for external databases query
2021-12-27 18:53:52 +03:00
kssenii
55dfaef4de Settings changes as key value 2021-12-27 17:45:00 +03:00
Kseniia Sumarokova
5fd952fe1b
Merge branch 'master' into materialized-postgresql-fix-cleanup 2021-12-27 17:25:51 +03:00
OnePiece
555b20bc2e
Update MergeTreePartsMover.cpp 2021-12-27 21:38:51 +08:00
OnePiece
47f0e3c72f
Update MergeTreePartsMover.cpp 2021-12-27 20:46:51 +08:00
kssenii
b90f53ffba Fix unit tests 2021-12-27 15:31:49 +03:00
vdimir
5c47229797
fix whitespace in StorageSystemClusters.cpp 2021-12-27 15:26:07 +03:00
Amos Bird
1cbd05ae9e
Correctly prevent nullable primary keys if necessary. 2021-12-27 17:32:37 +08:00
kssenii
a1aab3a82d Fix 2021-12-27 11:59:33 +03:00
Vladimir C
ea5f867289
Merge pull request #33179 from kssenii/materialized-postgresql-fix-detach-with-schema 2021-12-27 11:42:59 +03:00
Vladimir C
211a5a8d6b
Merge pull request #33159 from xuyatian/master 2021-12-27 11:40:59 +03:00
lgbo-ustc
0a39273fee fixed code style 2021-12-27 15:47:22 +08:00
taiyang-li
2d6eb40879 fix style 2021-12-27 15:31:24 +08:00
Ilya Yatsishin
a7cf7b4d6f
Merge pull request #33124 from bigo-sg/inject_gitinfo 2021-12-27 10:21:12 +03:00
taiyang-li
dd882085d0 Merge branch 'hive_table' of https://github.com/bigo-sg/ClickHouse into bigo_hive_table 2021-12-27 15:14:43 +08:00
taiyang-li
9036b18c2f merge master 2021-12-27 15:12:48 +08:00
lgbo-ustc
078521496a modifications based on pr review 2021-12-27 15:10:48 +08:00
kssenii
9f17e7bad0 Fix ddl validation 2021-12-26 20:56:31 +03:00
taiyang-li
8daeefeebe inject gitinfo 2021-12-26 15:46:38 +08:00
kssenii
2bcb122cd0 Clean up unused replication slots 2021-12-26 10:15:03 +03:00
kssenii
ef22ebad85 Better startup 2021-12-25 21:30:36 +03:00
kssenii
e663cb4574 Fix detaching-attaching table with schema 2021-12-25 19:51:55 +03:00
Kseniia Sumarokova
f8083450ab
Update ExternalDataSourceConfiguration.cpp 2021-12-25 17:43:00 +03:00
Kruglov Pavel
22dbaec11a
Merge pull request #33142 from Avogar/fix-s3-write
Don't allow to write into S3 if path contains globs
2021-12-25 16:07:38 +03:00
kssenii
59dd5ba063 Validate config keys for external dictionaries 2021-12-25 14:26:35 +03:00
taiyang-li
a88b2975c0 inject into system.build_options 2021-12-25 15:45:50 +08:00
Alexey Milovidov
f03cb4c762 Cleanup trash from Kafka and HDFS 2021-12-25 06:10:59 +03:00
Yatian Xu
565a38a2e3 fix incorrect metric: StorageBufferBytes 2021-12-24 17:49:59 -08:00
alexey-milovidov
888a5532a6
Merge pull request #33134 from ClickHouse/musl-check-2
Prepare ClickHouse to be built with musl-libc
2021-12-24 19:44:47 +03:00
avogar
58966e90e9 Don't allow to write into S3 if path contains globs 2021-12-24 17:13:35 +03:00
Nikita Mikhaylov
29fdc422d0
Merge pull request #32974 from nikitamikhaylov/hdfsCluster
Merging #32400
2021-12-24 16:02:54 +03:00
Alexey Milovidov
29d28c531f Move code around to avoid dlsym on Musl 2021-12-24 12:25:27 +03:00
Alexey Milovidov
204cc11e29 Fix incorrect include 2021-12-24 12:25:26 +03:00
Anton Popov
1f67177d4d fix optimize_read_in_order with preliminary merge and add some tests 2021-12-23 18:32:31 +03:00
lgbo-ustc
d29a4d0527 add trySet() in LRUCache 2021-12-23 16:16:33 +08:00
lgbo-ustc
ccc39bad80 move IRemoteFileMetadata info Storages/Cache 2021-12-23 15:56:33 +08:00
lgbo-ustc
a00ab6551b compile error 2021-12-23 15:43:42 +08:00
lgbo-ustc
f0b22d2882 fixed code-style 2021-12-23 15:15:39 +08:00
alexey-milovidov
0f8295de98
Merge pull request #33046 from ClickHouse/fix-prefetch-for-lc
Fix preferch for LowCardinality.
2021-12-23 08:56:16 +03:00
lgbo-ustc
e19fd5bc76 update some comments 2021-12-23 12:01:44 +08:00
lgbo-ustc
d205f9ecab codes refactor 2021-12-23 11:50:26 +08:00
Kseniia Sumarokova
639c2c1ce7
Merge pull request #32976 from Avogar/fix-hdfs-url-check
Fix hdfs url check
2021-12-22 23:57:49 +03:00
Kseniia Sumarokova
764bb1181f
Merge pull request #33017 from stigsb/table-overrides-drop-add-column-support
Remove the possibility of adding columns with table overrides
2021-12-22 22:30:21 +03:00
Dmitry Novik
1c83f31e51
Merge pull request #31885 from amosbird/optimizeprojectionmat
Optimize single part projection materialization
2021-12-22 19:36:40 +03:00
mergify[bot]
1e0642065b
Merge branch 'master' into deduplication_token_7461 2021-12-22 15:27:28 +00:00
Nikolai Kochetov
bb24a494ca Fix preferch for LowCardinality. 2021-12-22 15:26:16 +03:00
Stig Bakken
1cc70ad524 Remove the possibility of adding columns with table overrides
(except ALIAS columns)

Supporting this properly requires more comprehensive changes.
2021-12-22 08:20:14 +01:00
Kseniia Sumarokova
484a4d62dd
Merge pull request #32835 from kssenii/fising-s3
Fix optimization with lazy seek for async reads from remote fs
2021-12-22 09:38:23 +03:00
Anton Ivashkin
e88b97dafb Fix typos 2021-12-21 19:56:29 +03:00
Amos Bird
f00cc3b944
optimize single part projection materialization 2021-12-21 22:39:06 +08:00
ianton-ru
e6fd4bfb50
Merge branch 'master' into MDB-15474 2021-12-21 17:38:36 +03:00
Anton Ivashkin
33cbfc89ef Move logic for replicated part to StorageReplicatedMergeTree class 2021-12-21 17:29:50 +03:00
Anton Ivashkin
0c0bf66334 Merge master 2021-12-21 17:27:54 +03:00
lgbo-ustc
1f59ce760c typo 2021-12-21 19:06:00 +08:00
kssenii
ebcb66a361 Fix 2021-12-21 10:58:02 +03:00
taiyang-li
2597925724 merge master 2021-12-21 15:55:39 +08:00
taiyang-li
cff3c20742 Merge branch 'hive_table' of https://github.com/bigo-sg/ClickHouse into bigo_hive_table 2021-12-21 15:01:23 +08:00
taiyang-li
2fd63f9381 rename some symboles 2021-12-21 15:00:47 +08:00
taiyang-li
40aa359201 refactor storage hive 2021-12-21 14:44:31 +08:00
mergify[bot]
649226853d
Merge branch 'master' into fix-clang-tidy 2021-12-21 06:05:57 +00:00
lgbo-ustc
2e2354be40 1. add tryDel and getOrTrySet in LRUCache
2. fixed bugs in LRUCache for evicting elements
2021-12-21 10:39:59 +08:00
lgbo-ustc
2c67516432 1. fixed code style
2. fixed bugs in lru cache
2021-12-21 09:44:04 +08:00
kssenii
30996b2448 Fix 2021-12-21 01:19:51 +03:00
mergify[bot]
e21c04ff9c
Merge branch 'master' into fix-data-part-modification-time 2021-12-20 18:58:47 +00:00
Anton Ivashkin
c724b074ae Remove zero-copy version converter 2021-12-20 20:23:25 +03:00
Maksim Kita
dd0d3de050
Merge pull request #32970 from kitaisreal/loops-remove-postfix-increment
Loops remove postfix increment
2021-12-20 19:51:07 +03:00
Nikita Mikhaylov
d9c13900f5 Fix build + style 2021-12-20 14:31:56 +00:00
Maksim Kita
ebff389701
Merge pull request #32972 from kitaisreal/containers-iteration-fix-erase
Containers iteration fix erase
2021-12-20 16:47:19 +03:00
Anton Ivashkin
f0b9a4327a Use table UUID in zero-copy shared label in ZooKeeper 2021-12-20 16:19:00 +03:00
Zhichang Yu
869cdbeb44 added table function hdfsCluster 2021-12-20 13:14:44 +00:00
avogar
e31c18e9f8 Fix hdfs url check 2021-12-20 16:09:48 +03:00
Maksim Kita
51477adf1b Updated additional cases 2021-12-20 15:55:07 +03:00
Nikolai Kochetov
2e62f086a1
Merge pull request #32751 from ClickHouse/fix-32668
Apply some more optimizations to NO_QUERY ast.
2021-12-20 15:47:25 +03:00
lgbo-ustc
71353ceede refactor metadata class factory 2021-12-20 20:33:41 +08:00
Maksim Kita
3feab5a975 Containers iteration fix erase 2021-12-20 13:42:31 +03:00
taiyang-li
7192458601 fix code typo 2021-12-20 18:10:15 +08:00
save-my-heart
1abe978455 fix wrong modification_time after move part 2021-12-20 16:29:06 +08:00
Igor Nikonov
100ee92c64 insert_deduplication_token setting for INSERT statement
The setting allows a user to provide own deduplication semantic in Replicated*MergeTree
If provided, it's used instead of data digest to generate block ID
So, for example, by providing a unique value for the setting in each INSERT statement,
user can avoid the same inserted data being deduplicated

Inserting data within the same INSERT statement are split into blocks
according to the *insert_block_size* settings
(max_insert_block_size, min_insert_block_size_rows, min_insert_block_size_bytes).
Each block with the same INSERT statement will get an ordinal number.
The ordinal number is added to insert_deduplication_token to get block dedup token
i.e. <token>_0, <token>_1, ... Deduplication is done per block
So, to guarantee deduplication for two same INSERT queries,
dedup token and number of blocks to have to be the same

Issue: #7461
2021-12-19 13:15:45 +00:00
kssenii
e6ebb55c4e Move to Interpreters 2021-12-17 22:19:10 +03:00
kssenii
dd902b6875 Merge remote-tracking branch 'upstream/master' into table-override-pg 2021-12-17 22:03:58 +03:00
Nikolai Kochetov
93a33d52dc Merge branch 'fix-32668' of github.com:yandex/ClickHouse into fix-32668 2021-12-17 20:37:29 +03:00
Nikolai Kochetov
c8a92c046f Another try 2021-12-17 20:36:37 +03:00
Kseniia Sumarokova
f6e7e11742
Merge pull request #32907 from kssenii/url-better
Better handling of globs for url storage
2021-12-17 19:08:38 +03:00
kssenii
40c266840c Fixes 2021-12-17 18:16:29 +03:00
alesapin
6bd7e425c6
Merge pull request #22535 from CurtizJ/sparse-serialization
Sparse serialization and ColumnSparse
2021-12-17 15:26:17 +03:00
kssenii
b1794a47c3 StorageURL improve 2021-12-17 14:03:37 +03:00
Kseniia Sumarokova
77b3ebe44e
Merge pull request #32734 from kssenii/mysql-fix-timeouts
Allow to control connection timeouts for mysql
2021-12-17 10:13:33 +03:00
taiyang-li
d033fc4c24 merge master and fix conflict 2021-12-17 15:11:21 +08:00
kssenii
97222be3d9 Merge remote-tracking branch 'upstream/master' into table-override-pg 2021-12-17 01:14:28 +03:00
mergify[bot]
4722bc6649
Merge branch 'master' into fix_skipped_mutations 2021-12-16 13:25:00 +00:00
Maksim Kita
858d9f0f5f StorageLiveView fix function style 2021-12-16 12:29:20 +03:00
Maksim Kita
33374f8b0a
Merge pull request #31062 from Enmk/Governance/view_comment
Views with comment
2021-12-16 12:27:12 +03:00
kssenii
7583c8007e Some review fixes 2021-12-16 00:38:46 +03:00
Alexander Tokmakov
1b02136941 fix skipping of some mutations 2021-12-15 21:19:29 +03:00
kssenii
dc1f0c58fa Merge remote-tracking branch 'upstream/master' into table-override-pg 2021-12-15 15:27:58 +03:00
Anton Popov
ec46cbef20
Merge pull request #32756 from CurtizJ/fix-index-hypothesis
Fix race in skipping index of type `hypothesis`
2021-12-15 15:26:35 +03:00
kssenii
2e813fe3cd Update defaults 2021-12-15 08:18:43 +00:00
Anton Popov
2e388a72da
Merge pull request #32742 from nautaa/flush_in_memory_part
flush all InMemoryDataParts when wal is not enabled
2021-12-14 23:09:10 +03:00
Kseniia Sumarokova
bf415378be
Merge pull request #32607 from Vxider/watch-events-supports
Events clause support for Window View watch query
2021-12-14 23:08:24 +03:00
kssenii
47f549166d Table Override for PostgreSQL 2021-12-14 20:36:43 +03:00
Anton Popov
16312e7e4a Merge remote-tracking branch 'upstream/master' into HEAD 2021-12-14 18:58:17 +03:00
Anton Popov
84d6715e3d fix race in skipping index of type hypothesis 2021-12-14 18:28:55 +03:00
Anton Popov
96070b414d support read_in_order optimization if prefix of sorting key is fixed 2021-12-14 15:54:20 +03:00
nautaa
a96da58360 fix 2021-12-14 20:11:43 +08:00
nautaa
5545793555 Merge branch 'flush_in_memory_part' of github.com:nautaa/ClickHouse into flush_in_memory_part 2021-12-14 19:42:15 +08:00
nautaa
3469fb8165 fix 2021-12-14 19:40:35 +08:00
mergify[bot]
60a864f91c
Merge branch 'master' into flush_in_memory_part 2021-12-14 11:14:21 +00:00
Anton Popov
bbb981655d
Merge pull request #32695 from CurtizJ/fix-add-ttl
Fix `MATERIALIZE TTL` with `TTL WHERE` and `TTL GROUP BY`
2021-12-14 14:01:15 +03:00
taiyang-li
a0b90fa9ec fix bugs and add some docs 2021-12-14 18:44:15 +08:00
taiyang-li
52c118856f Merge branch 'hive_table' of https://github.com/bigo-sg/ClickHouse into bigo_hive_table 2021-12-14 17:37:45 +08:00
taiyang-li
ca3f7425a4 fix code 2021-12-14 17:37:31 +08:00
nautaa
0da3881ff4 flush all InMemoryDataParts when wal is not enabled 2021-12-14 16:31:17 +08:00
lgbo-ustc
248e933fc4 Add test for local cache 2021-12-14 16:06:30 +08:00
mergify[bot]
a5599d9fe2
Merge branch 'master' into watch-events-supports 2021-12-14 06:15:58 +00:00