ClickHouse/src/Storages
Han Fei b7eef62458
Merge pull request #45491 from azat/dist/async-send-refactoring
[RFC] Rewrite distributed sends to avoid using filesystem as a queue, use in-memory queue instead
2023-03-06 12:32:33 +01:00
..
Cache
Distributed Merge pull request #45491 from azat/dist/async-send-refactoring 2023-03-06 12:32:33 +01:00
examples
FileLog Apply some CTAD 2023-03-02 13:36:47 +00:00
fuzzers
HDFS Merge remote-tracking branch 'upstream/master' into resubmit-prefetches 2023-02-11 11:36:23 +01:00
Hive Apply some CTAD 2023-03-02 13:36:47 +00:00
Kafka Merge pull request #45693 from Avogar/fix-kafka-protobuf 2023-03-02 11:50:45 +01:00
LiveView Fixed code review issues 2023-02-18 17:06:00 +01:00
MeiliSearch forbid runtime strings 2023-01-26 10:52:47 +01:00
MergeTree Merge pull request #47154 from azat/prewhere-columns-fix 2023-03-06 10:03:31 +01:00
MySQL Better alias definition 2023-02-01 13:54:03 +00:00
NATS Merge pull request #46337 from kssenii/rabbitmq-tests-fix-2 2023-02-14 11:44:34 +01:00
PostgreSQL Merge remote-tracking branch 'upstream/master' into return-materialized-postrgesql-tests-back 2023-02-20 14:37:39 +01:00
RabbitMQ rabbitmq-test-fix 2023-02-20 14:16:40 +01:00
RocksDB Better 2023-02-13 15:23:48 +00:00
System Rename StorageDistributedDirectoryMonitor to DistributedAsyncInsertDirectoryQueue 2023-02-28 22:33:36 +01:00
tests Less stupid tests 2023-01-11 15:20:18 +01:00
WindowView Merge pull request #46471 from kitaisreal/analyzer-planner-fixes-before-enable-by-default 2023-02-19 13:49:18 +03:00
addColumnsStructureToQueryWithClusterEngine.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
addColumnsStructureToQueryWithClusterEngine.h
AlterCommands.cpp fix mutations with sparse columns 2023-02-21 15:13:29 +00:00
AlterCommands.h remove useless code 2023-01-25 16:11:15 +00:00
checkAndGetLiteralArgument.cpp s390x build support 2023-01-20 21:16:55 +00:00
checkAndGetLiteralArgument.h
CheckResults.h
CMakeLists.txt
ColumnDefault.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ColumnDefault.h dedicated flag for ephemeral default, make ephemeral accept expression instead of literal 2022-12-11 00:40:35 +00:00
ColumnDependency.h
ColumnsDescription.cpp Analyzer matcher resolve fix 2023-02-16 12:17:02 +01:00
ColumnsDescription.h Analyzer matcher resolve fix 2023-02-16 12:17:02 +01:00
CompressionCodecSelector.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ConstraintsDescription.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ConstraintsDescription.h
DataDestinationType.h
ExecutableSettings.cpp
ExecutableSettings.h Better alias definition 2023-02-01 13:54:03 +00:00
ExternalDataSourceConfiguration.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ExternalDataSourceConfiguration.h Merge branch 'master' into add-support-for-settings-alias 2023-02-09 16:07:45 +01:00
extractKeyExpressionList.cpp
extractKeyExpressionList.h
Freeze.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
Freeze.h
getStructureOfRemoteTable.cpp Parallel replicas. Part [2] (#43772) 2023-02-03 14:34:18 +01:00
getStructureOfRemoteTable.h
getVirtualsForStorage.cpp
getVirtualsForStorage.h
IMessageProducer.cpp Fix race 2023-02-13 16:02:41 +01:00
IMessageProducer.h Fix race 2023-02-13 16:02:41 +01:00
IndicesDescription.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
IndicesDescription.h
IStorage_fwd.h
IStorage.cpp Revert "Revert "Merge pull request #45493 from azat/fix-detach"" 2023-01-24 14:15:51 +01:00
IStorage.h Remove PVS-Studio 2023-02-19 23:30:05 +01:00
IStorageCluster.h Parallel distributed insert select with s3Cluster [3] (#44955) 2023-01-09 13:30:32 +01:00
IStorageDataLake.h fix 2023-02-17 12:27:53 +00:00
KeyDescription.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
KeyDescription.h
KVStorageUtils.cpp Check type of prepared set 2023-01-12 10:44:55 +00:00
KVStorageUtils.h
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h
MemorySettings.cpp
MemorySettings.h Better alias definition 2023-02-01 13:54:03 +00:00
MessageQueueSink.cpp Add logging 2023-02-10 18:14:49 +01:00
MessageQueueSink.h Don't use WithContext to prevent context expiration 2022-12-21 10:43:48 +00:00
MutationCommands.cpp Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
MutationCommands.h Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
NamedCollectionsHelpers.cpp Load named collections on first access 2023-02-20 16:33:45 +01:00
NamedCollectionsHelpers.h Merge remote-tracking branch 'upstream/master' into use-new-named-collections-code-2 2023-01-03 16:41:30 +01:00
PartitionCommands.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
PartitionCommands.h
PartitionedSink.cpp
PartitionedSink.h
ProjectionsDescription.cpp forbid old ctor of Exception 2023-01-23 22:18:05 +01:00
ProjectionsDescription.h
ReadFinalForExternalReplicaStorage.cpp
ReadFinalForExternalReplicaStorage.h
ReadFromStorageProgress.cpp Fix build 2022-12-12 12:06:00 +01:00
ReadFromStorageProgress.h
ReadInOrderOptimizer.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ReadInOrderOptimizer.h
registerStorages.cpp fix 2023-02-19 03:47:14 +00:00
registerStorages.h
RenamingRestrictions.h Minor code polishing 2022-12-22 14:31:42 +01:00
S3DataLakeMetadataReadHelper.cpp fix 2023-02-15 03:56:01 +00:00
S3DataLakeMetadataReadHelper.h fix 2023-02-15 03:56:01 +00:00
SelectQueryDescription.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
SelectQueryDescription.h Analyzer support LiveView 2023-02-16 12:17:03 +01:00
SelectQueryInfo.h Parallel replicas. Part [2] (#43772) 2023-02-03 14:34:18 +01:00
SetSettings.cpp
SetSettings.h Better alias definition 2023-02-01 13:54:03 +00:00
StorageBuffer.cpp Remove PVS-Studio 2023-02-19 23:30:05 +01:00
StorageBuffer.h [RFC] Replacing merge tree new engine (#41005) 2023-02-16 16:03:16 +03:00
StorageConfiguration.h Simplify code around storage s3 configuration 2023-02-06 16:23:17 +01:00
StorageDeltaLake.cpp fix 2023-02-16 12:48:22 +00:00
StorageDeltaLake.h refactor 2023-02-17 08:27:52 +00:00
StorageDictionary.cpp Apply some CTAD 2023-03-02 13:36:47 +00:00
StorageDictionary.h
StorageDistributed.cpp Merge pull request #45491 from azat/dist/async-send-refactoring 2023-03-06 12:32:33 +01:00
StorageDistributed.h Fix race for distributed sends from disk 2023-02-28 22:33:36 +01:00
StorageDummy.cpp Fixed tests 2023-03-01 18:05:07 +01:00
StorageDummy.h Fixed tests 2023-03-01 18:05:07 +01:00
StorageExecutable.cpp Avoid std::out_of_range exception in StorageExecutable 2022-12-28 16:21:58 +00:00
StorageExecutable.h
StorageExternalDistributed.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageExternalDistributed.h
StorageFactory.cpp Remove PVS-Studio 2023-02-19 23:30:05 +01:00
StorageFactory.h
StorageFile.cpp Separate out DirectoryMonitorSource as DistributedAsyncInsertSource 2023-02-28 22:33:36 +01:00
StorageFile.h review fixes 2023-01-12 15:51:04 +00:00
StorageGenerateRandom.cpp GenerateRandom: Add support for maps 2023-01-30 15:23:22 +01:00
StorageGenerateRandom.h
StorageHudi.cpp fix 2023-02-16 12:48:22 +00:00
StorageHudi.h refactor 2023-02-17 08:27:52 +00:00
StorageIceberg.cpp fix build without avro 2023-02-18 17:32:35 +00:00
StorageIceberg.h fix typo 2023-02-18 17:34:37 +00:00
StorageInMemoryMetadata.cpp Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
StorageInMemoryMetadata.h Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
StorageInput.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageInput.h
StorageJoin.cpp Remove PVS-Studio 2023-02-19 23:30:05 +01:00
StorageJoin.h force_wait flag is no longer needed 2023-01-30 18:38:28 +01:00
StorageKeeperMap.cpp Better 2023-02-13 15:23:48 +00:00
StorageKeeperMap.h Add support for DELETE/UPDATE for KeeperMap 2023-02-13 11:22:43 +00:00
StorageLog.cpp Merge branch 'master' into fix_insert_cancellation_in_native_protocol 2023-02-23 22:35:31 +01:00
StorageLog.h review fixes 2023-01-12 15:51:04 +00:00
StorageLogSettings.cpp fix 2023-02-11 13:58:19 +00:00
StorageLogSettings.h Storage Log faminy support settings storage policy 2023-02-04 14:28:31 +00:00
StorageMaterializedMySQL.cpp
StorageMaterializedMySQL.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageMaterializedView.cpp Merge branch 'master' into parameters-rename-query 2023-02-17 17:30:37 +01:00
StorageMaterializedView.h [RFC] Replacing merge tree new engine (#41005) 2023-02-16 16:03:16 +03:00
StorageMemory.cpp force_wait flag is no longer needed 2023-01-30 18:38:28 +01:00
StorageMemory.h force_wait flag is no longer needed 2023-01-30 18:38:28 +01:00
StorageMerge.cpp Fixed tests 2023-03-01 18:05:07 +01:00
StorageMerge.h Fixed tests 2023-03-01 18:05:07 +01:00
StorageMergeTree.cpp Apply some CTAD 2023-03-02 13:36:47 +00:00
StorageMergeTree.h Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
StorageMongoDB.cpp Move MongoDB and PostgreSQL sources to Sources folder 2023-02-14 22:35:10 +00:00
StorageMongoDB.h Replace old named collections code for mongo 2022-12-17 00:50:25 +01:00
StorageMongoDBSocketFactory.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageMongoDBSocketFactory.h
StorageMySQL.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageMySQL.h
StorageNull.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageNull.h
StoragePostgreSQL.cpp Move MongoDB and PostgreSQL sources to Sources folder 2023-02-14 22:35:10 +00:00
StoragePostgreSQL.h Replace old named collections code for postgresql 2022-12-17 01:30:55 +01:00
StorageProxy.h [RFC] Replacing merge tree new engine (#41005) 2023-02-16 16:03:16 +03:00
StorageReplicatedMergeTree.cpp Merge branch 'master' into fix_insert_cancellation_in_native_protocol 2023-03-01 14:26:41 +01:00
StorageReplicatedMergeTree.h Revert "Fix several RENAME COLUMN bugs." 2023-02-26 05:47:05 +03:00
StorageS3.cpp fix 2023-02-17 12:42:24 +00:00
StorageS3.h fix 2023-02-17 12:42:24 +00:00
StorageS3Cluster.cpp fix 2023-02-17 12:42:24 +00:00
StorageS3Cluster.h Simplify code around storage s3 configuration 2023-02-06 16:23:17 +01:00
StorageS3Settings.cpp Merge branch 'master' into s3-client-polish 2023-02-08 10:07:00 +00:00
StorageS3Settings.h Merge branch 'master' into s3-client-polish 2023-02-08 10:07:00 +00:00
StorageSet.cpp Fix bug in tables drop which can lead to potential query hung 2023-01-30 17:00:28 +01:00
StorageSet.h
StorageSnapshot.cpp Added is_create_parameterized_view flag in ActionsVisitor, added functions for column manipulation for parameterized view in StorageView, updated tests to fix flaky test issues and addressed small review comments- 40907 Parameterized views as table functions 2023-01-03 11:00:39 +01:00
StorageSnapshot.h
StorageSQLite.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageSQLite.h
StorageStripeLog.cpp Storage Log faminy support settings storage policy 2023-02-04 14:28:31 +00:00
StorageStripeLog.h review fixes 2023-01-12 15:51:04 +00:00
StorageTableFunction.h Analyzer added distributed table functions support 2023-02-16 12:17:03 +01:00
StorageURL.cpp Pass headers from StorageURL to WriteBufferFromHTTP (#46996) 2023-03-03 13:55:52 +01:00
StorageURL.h Pass headers from StorageURL to WriteBufferFromHTTP (#46996) 2023-03-03 13:55:52 +01:00
StorageValues.cpp
StorageValues.h
StorageView.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageView.h Added is_create_parameterized_view flag in ActionsVisitor, added functions for column manipulation for parameterized view in StorageView, updated tests to fix flaky test issues and addressed small review comments- 40907 Parameterized views as table functions 2023-01-03 11:00:39 +01:00
StorageXDBC.cpp Reduce redundancies 2023-02-07 12:27:23 +00:00
StorageXDBC.h
TableLockHolder.h
transformQueryForExternalDatabase.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
transformQueryForExternalDatabase.h
TTLDescription.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
TTLDescription.h
TTLMode.h
UVLoop.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
VirtualColumnUtils.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
VirtualColumnUtils.h