ClickHouse/src/Storages
Alexey Milovidov 354cf09892
Merge pull request #48549 from cangyin/add_zookeeper_name
add zookeeper name in system.replicas
2023-04-27 01:22:22 +03:00
..
Cache Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
DataLakes Fix building iceberg without avro 2023-04-25 08:42:42 +02:00
Distributed Fix build, pt. II 2023-04-05 11:23:09 +00:00
examples Refactor 2023-03-23 08:58:56 +00:00
FileLog Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
fuzzers
HDFS Merge branch 'master' into non-blocking-connect 2023-04-19 13:39:40 +02:00
Hive Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
Kafka Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
LiveView Remove -Wshadow suppression which leaked into global namespace 2023-04-13 08:46:40 +00:00
MeiliSearch fix stdlib compatibility issues 2023-03-29 12:20:27 +03:00
MergeTree Merge pull request #48549 from cangyin/add_zookeeper_name 2023-04-27 01:22:22 +03:00
MySQL Fox 2023-04-13 19:36:25 +02:00
NATS Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
PostgreSQL Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
RabbitMQ Merge pull request #48845 from kssenii/fix-data-race-in-rabbitmq 2023-04-20 11:43:18 +02:00
RocksDB Merge pull request #48435 from ClickHouse/kv-update-only-affected-rows 2023-04-06 12:33:54 +02:00
System Merge branch 'master' into add_zookeeper_name 2023-04-26 13:53:57 +03:00
tests Remove unused mockSystemDatabase from gtest_transform_query_for_external_database 2023-03-29 11:02:50 +00:00
WindowView use Poco::Timestamp() instead of std::time 2023-04-20 14:36:54 +00: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 prohibit DEFAULT/EPHEMERAL/ALIAS in KafkaEngine 2023-03-03 16:25:42 +01:00
ColumnsDescription.h Docs: Small cleanups after Kafka fix #47138 2023-03-07 19:50:42 +00:00
CompressionCodecSelector.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ConstraintsDescription.cpp Add support for substitute column 2023-03-17 13:38:01 +00:00
ConstraintsDescription.h Add support for substitute column 2023-03-17 13:38:01 +00:00
DataDestinationType.h
ExecutableSettings.cpp
ExecutableSettings.h Better alias definition 2023-02-01 13:54:03 +00:00
ExternalDataSourceConfiguration.cpp Remove redundant 2023-02-27 12:32:13 +01:00
ExternalDataSourceConfiguration.h Finish for streaming engines 2023-02-21 14:50:55 +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 fix skip indexes 2023-03-06 15:29:13 +00:00
IndicesDescription.h
IStorage_fwd.h
IStorage.cpp Setting parallelize_output_from_storages 2023-04-24 12:22:45 +00:00
IStorage.h Update src/Storages/IStorage.h 2023-04-21 11:50:07 +03:00
IStorageCluster.h Parallel distributed insert select with s3Cluster [3] (#44955) 2023-01-09 13:30:32 +01: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 Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h Merge remote-tracking branch 'origin/master' into rs/qc-cachebase 2023-03-15 10:55:53 +00:00
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 "Revert "Fix several RENAME COLUMN bugs."" 2023-02-27 12:27:57 +01:00
MutationCommands.h Revert "Revert "Fix several RENAME COLUMN bugs."" 2023-02-27 12:27:57 +01:00
NamedCollectionsHelpers.cpp Review fixes 2023-03-17 13:56:02 +01:00
NamedCollectionsHelpers.h Fix clang-tidy 2023-03-05 22:12:51 +01:00
PartitionCommands.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
PartitionCommands.h
PartitionedSink.cpp
PartitionedSink.h Fix double whitespace in exception message 2023-04-15 23:54:10 +02:00
ProjectionsDescription.cpp forbid old ctor of Exception 2023-01-23 22:18:05 +01:00
ProjectionsDescription.h
ReadFinalForExternalReplicaStorage.cpp Fixes for -Wshorten-64-to-32 2022-10-21 13:25:19 +02:00
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 build without parquet 2023-04-17 21:37:32 -07:00
registerStorages.h
removeGroupingFunctionSpecializations.cpp Resolve as FunctionGrouping 2023-03-14 03:33:31 +00:00
removeGroupingFunctionSpecializations.h Move visitor 2023-03-10 14:36:56 +00:00
RenamingRestrictions.h Minor code polishing 2022-12-22 14:31:42 +01: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 Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
SetSettings.cpp
SetSettings.h Better alias definition 2023-02-01 13:54:03 +00:00
StorageBuffer.cpp Only check MV on ALTER when necessary 2023-03-27 17:45:15 +02: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
StorageDictionary.cpp Apply some CTAD 2023-03-02 13:36:47 +00:00
StorageDictionary.h Better way to define for which storage output is parallelized 2023-04-14 12:00:02 +00:00
StorageDistributed.cpp Merge branch 'master' into non-blocking-connect 2023-04-19 13:39:40 +02:00
StorageDistributed.h Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02: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 Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
StorageExternalDistributed.cpp Review fixes 2023-03-17 13:56:02 +01:00
StorageExternalDistributed.h Replace for table function remote, and external storage 2023-02-21 14:33:37 +01:00
StorageFactory.cpp Remove PVS-Studio 2023-02-19 23:30:05 +01:00
StorageFactory.h
StorageFile.cpp Use generic way to parallelize output for file() 2023-04-15 12:35:24 +00:00
StorageFile.h Merge pull request #48612 from ClickHouse/remove-strange-code-2 2023-04-11 06:17:45 +03:00
StorageGenerateRandom.cpp Fix IBM 2023-04-21 12:38:45 +02:00
StorageGenerateRandom.h Fix SipHash integer hashing and byte order issue in GenerateRandom for s390x 2023-03-14 11:54:17 -04:00
StorageInMemoryMetadata.cpp Revert "Revert "Fix several RENAME COLUMN bugs."" 2023-02-27 12:27:57 +01:00
StorageInMemoryMetadata.h Revert "Revert "Fix several RENAME COLUMN bugs."" 2023-02-27 12:27:57 +01:00
StorageInput.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageInput.h
StorageJoin.cpp Fix build 2023-03-13 10:49:51 +00:00
StorageJoin.h Fix build 2023-03-13 10:49:51 +00:00
StorageKeeperMap.cpp Merge pull request #48435 from ClickHouse/kv-update-only-affected-rows 2023-04-06 12:33:54 +02:00
StorageKeeperMap.h Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
StorageLog.cpp Add ability to throttle BACKUPs on per-server/backup basis 2023-04-05 09:39:48 +02: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 Fix race between DROP MatView and RESTART REPLICAS (#47863) 2023-04-01 15:26:00 +03:00
StorageMaterializedView.h [RFC] Replacing merge tree new engine (#41005) 2023-02-16 16:03:16 +03:00
StorageMemory.cpp Merge pull request #48219 from ClickHouse/vdimir/lock_order_47072 2023-04-06 11:54:33 +02:00
StorageMemory.h add optimization for MemoryStorageStep 2023-03-19 20:35:24 +01:00
StorageMerge.cpp Merge pull request #48062 from Algunenano/unnecessary_alter_checks 2023-04-03 17:23:11 -04:00
StorageMerge.h Fix crash in EXPLAIN PIPELINE for Merge over Distributed 2023-04-02 10:48:23 +02:00
StorageMergeTree.cpp Merge branch 'master' into feature/support_settings_allow_suspicious_indices 2023-04-23 10:08:00 +08:00
StorageMergeTree.h Caches for multiple mutations 2023-04-14 16:12:19 +02:00
StorageMongoDB.cpp Add support for {server_uuid} macro 2023-04-09 03:04:26 +02:00
StorageMongoDB.h Replace for table function remote, and external storage 2023-02-21 14:33:37 +01:00
StorageMongoDBSocketFactory.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
StorageMongoDBSocketFactory.h
StorageMySQL.cpp Fox 2023-04-13 19:36:25 +02:00
StorageMySQL.h Fix test_storage_mysql/test.py::test_settings_connection_wait_timeout 2023-04-14 20:10:57 +00:00
StorageNull.cpp Only check MV on ALTER when necessary 2023-03-27 17:45:15 +02:00
StorageNull.h Use generic way to parallelize output for file() 2023-04-15 12:35:24 +00:00
StoragePostgreSQL.cpp fix prepared statement insert already exists (#49154) 2023-04-26 14:06:10 +03:00
StoragePostgreSQL.h Finish for mysql 2023-02-20 21:37:38 +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 add_zookeeper_name 2023-04-26 13:53:57 +03:00
StorageReplicatedMergeTree.h Fix unexpected part name error when trying to drop a ignored detache partition with zero copy replication 2023-04-17 15:57:41 +00:00
StorageS3.cpp Merge pull request #48727 from ClickHouse/parallel-processing-from-storages 2023-04-23 23:10:32 +03:00
StorageS3.h Better 2023-04-17 04:58:32 +00:00
StorageS3Cluster.cpp Merge branch 'master' into non-blocking-connect 2023-04-19 13:39:40 +02:00
StorageS3Cluster.h Better 2023-03-28 18:57:24 +02:00
StorageS3Settings.cpp Remove quirk for s3_max_single_part_upload_size (when it exceed INT_MAX) 2023-04-17 10:49:55 +02:00
StorageS3Settings.h Merge branch 'master' into s3-client-polish 2023-02-08 10:07:00 +00:00
StorageSet.cpp Merge remote-tracking branch 'origin/master' into optimize-compilation 2023-04-13 16:04:09 +02:00
StorageSet.h Improve file includes 2023-03-24 03:44:52 +01:00
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 Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
StorageSQLite.h
StorageStripeLog.cpp Add ability to throttle BACKUPs on per-server/backup basis 2023-04-05 09:39:48 +02: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 Better 2023-04-17 04:58:32 +00:00
StorageURL.h Merge remote-tracking branch 'upstream/master' into named-collections-finish 2023-03-17 13:02:36 +01:00
StorageValues.cpp
StorageValues.h
StorageView.cpp Added space to if expression of replaceQueryParameterWithValue function 2023-03-14 09:26:53 +01:00
StorageView.h Addressed review comments for parameterized view bug fix 2023-03-14 09:23:12 +01:00
StorageXDBC.cpp Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
StorageXDBC.h
TableLockHolder.h
transformQueryForExternalDatabase.cpp Better transformQueryForExternalDatabase for analyzer 2023-03-22 16:58:34 +00:00
transformQueryForExternalDatabase.h Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
transformQueryForExternalDatabaseAnalyzer.cpp Better transformQueryForExternalDatabase for analyzer 2023-03-22 16:58:34 +00:00
transformQueryForExternalDatabaseAnalyzer.h Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
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