ClickHouse/src/Storages
2024-02-18 11:04:01 +01:00
..
Cache Updated implementation 2024-01-25 14:31:49 +03:00
DataLakes Merge branch 'master' of github.com:ClickHouse/ClickHouse into auto-format-detection 2024-01-29 15:45:18 +00:00
Distributed Updated implementation 2024-01-25 14:31:49 +03:00
examples
FileLog Fix long shutdown of FileLog storage 2024-02-12 11:52:09 +01:00
fuzzers fix fuzzers, cmake refactor, add target fuzzers 2023-09-01 14:20:50 +00:00
HDFS Merge branch 'master' into auto-format-detection 2024-02-13 19:08:33 +01:00
Hive ActionsDAG buildFilterActionsDAG refactoring 2024-01-25 18:24:14 +03:00
Kafka Updated implementation 2024-01-25 14:31:49 +03:00
LiveView Updated implementation 2024-01-25 14:31:49 +03:00
MaterializedView Updated implementation 2024-01-25 14:31:49 +03:00
MergeTree Apply review comments 2024-02-18 05:13:17 +01:00
MySQL Intorduce *List definition for muli enum settings 2023-11-28 19:09:02 +00:00
NATS Add support for NATS credentials file 2024-02-02 23:34:01 +01:00
PostgreSQL Fix tidy 2024-02-06 20:34:12 +01:00
RabbitMQ Fix lazy initialization in RabbitMQ, fix possible deadlock on insert into unitialized queue engine 2024-01-29 20:09:09 +00:00
RocksDB Add comments to all columns of system tables (#58356) 2024-01-12 16:39:22 +01:00
S3Queue Merge pull request #59092 from Avogar/auto-format-detection 2024-02-16 14:32:18 +01:00
Statistics rename some code 2023-11-28 16:32:47 +01:00
System Revert "Implement system.dns_cache table (#59856)" 2024-02-17 01:46:41 +01:00
tests add test 2023-12-15 15:29:15 +00:00
WindowView Updated implementation 2024-01-25 14:31:49 +03:00
AlterCommands.cpp Merge remote-tracking branch 'ClickHouse/master' into column_level_compress_block 2024-01-18 19:12:57 +00:00
AlterCommands.h Some fixups 2024-01-18 20:49:50 +00:00
BlockNumberColumn.cpp Added a new column _block_number (#47532) 2023-09-20 11:31:12 +02:00
BlockNumberColumn.h Added a new column _block_number (#47532) 2023-09-20 11:31:12 +02:00
buildQueryTreeForShard.cpp Support non global in mode. 2024-02-05 17:05:21 +00:00
buildQueryTreeForShard.h Support non global in mode. 2024-02-05 17:05:21 +00:00
checkAndGetLiteralArgument.cpp add more check + line break 2023-07-05 15:04:38 +00:00
checkAndGetLiteralArgument.h
CheckResults.h Improve CHECK TABLE system query 2023-10-23 09:35:26 +00:00
CMakeLists.txt
ColumnDefault.cpp
ColumnDefault.h
ColumnDependency.h use statistic to order prewhere conditions better 2023-08-09 22:57:49 +02:00
ColumnsDescription.cpp Merge remote-tracking branch 'ClickHouse/master' into column_level_compress_block 2024-01-18 19:12:57 +00:00
ColumnsDescription.h Merge remote-tracking branch 'ClickHouse/master' into column_level_compress_block 2024-01-18 19:12:57 +00:00
CompressionCodecSelector.h
ConstraintsDescription.cpp Remove default value for argument 'ignore_aliases' from IAST::getTreeHash() 2023-11-13 10:27:38 +00:00
ConstraintsDescription.h
DataDestinationType.h
ExecutableSettings.cpp
ExecutableSettings.h StorageExecutable check_exit_code default to false 2023-08-18 15:38:47 +08:00
ExternalDataSourceConfiguration.cpp Remove broken lockless variant of re2 2023-09-14 16:40:42 +00:00
ExternalDataSourceConfiguration.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
extractKeyExpressionList.cpp
extractKeyExpressionList.h
extractTableFunctionArgumentsFromSelectQuery.cpp Fix schema inference with named collection, refactor Cluster table functions 2023-05-12 13:58:45 +00:00
extractTableFunctionArgumentsFromSelectQuery.h Fix schema inference with named collection, refactor Cluster table functions 2023-05-12 13:58:45 +00:00
Freeze.cpp Updated implementation 2024-01-25 14:31:49 +03:00
Freeze.h Updated implementation 2024-01-25 14:31:49 +03:00
getStructureOfRemoteTable.cpp Handle clusterAllReplicas/remote cases to avoid unnecessary logging 2023-09-12 12:52:29 +00:00
getStructureOfRemoteTable.h
IMessageProducer.cpp Fix lazy initialization in RabbitMQ, fix possible deadlock on insert into unitialized queue engine 2024-01-29 20:09:09 +00:00
IMessageProducer.h Updated implementation 2024-01-25 14:31:49 +03:00
IndicesDescription.cpp Revert "Revert "Support use alias column in indices"" 2023-12-05 18:22:32 +00:00
IndicesDescription.h Refactor IHints 2023-09-10 03:48:12 +00:00
IStorage_fwd.h
IStorage.cpp Merge branch 'master' into remove-old-projections-code 2023-12-20 17:45:38 +01:00
IStorage.h Check if I can remove KeyCondition analysis on AST. 2024-01-03 17:50:46 +00:00
IStorageCluster.cpp Merge branch 'master' into auto-format-detection 2024-01-26 15:51:35 +01:00
IStorageCluster.h Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
KeyDescription.cpp
KeyDescription.h
KVStorageUtils.cpp ActionsDAG buildFilterActionsDAG refactoring 2024-01-25 18:24:14 +03:00
KVStorageUtils.h Fix rocksdb with analyzer. 2023-11-06 18:46:39 +00:00
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h Better parameter name 2023-08-22 15:43:13 +00:00
MemorySettings.cpp
MemorySettings.h
MessageQueueSink.cpp Updated implementation 2024-01-25 14:31:49 +03:00
MessageQueueSink.h release buffers with exception context 2023-06-22 13:00:13 +02:00
MutationCommands.cpp Support query parameters in ALTER TABLE ... PART 2024-01-15 14:16:54 +03:00
MutationCommands.h add mutation command to apply deleted mask 2023-12-01 19:12:05 +00:00
NamedCollectionsHelpers.cpp Update src/Storages/NamedCollectionsHelpers.cpp 2023-11-21 11:14:48 +01:00
NamedCollectionsHelpers.h Fix style 2024-01-08 08:50:34 +00:00
PartitionCommands.cpp Support query parameters in ALTER TABLE ... PART 2024-01-15 14:16:54 +03:00
PartitionCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
PartitionedSink.cpp Minor changes 2023-11-22 12:13:47 +01:00
PartitionedSink.h release buffers with exception context 2023-06-22 13:00:13 +02:00
prepareReadingFromFormat.cpp Fix tests 2023-07-06 17:47:01 +00:00
prepareReadingFromFormat.h Fix tests 2023-07-05 17:56:03 +00:00
ProjectionsDescription.cpp Disable transform_null_in as well just in case. 2023-11-29 17:30:22 +00:00
ProjectionsDescription.h Refactor IHints 2023-09-10 03:48:12 +00:00
ReadFinalForExternalReplicaStorage.cpp remove old code of projection analysis 2023-10-12 23:25:20 +00:00
ReadFinalForExternalReplicaStorage.h
ReadInOrderOptimizer.cpp
ReadInOrderOptimizer.h
RedisCommon.cpp fix code style 2023-06-02 10:05:54 +08:00
RedisCommon.h fix build error for dwrwin 2023-06-02 10:05:54 +08:00
registerStorages.cpp Follow-up to #56490: Fix build with -DENABLE_LIBRARIES=0 2023-11-28 19:59:43 +00:00
registerStorages.h
removeGroupingFunctionSpecializations.cpp
removeGroupingFunctionSpecializations.h
RenamingRestrictions.h
ReplaceAliasByExpressionVisitor.cpp Revert "Revert "Support use alias column in indices"" 2023-12-05 18:22:32 +00:00
ReplaceAliasByExpressionVisitor.h Revert "Revert "Support use alias column in indices"" 2023-12-05 18:22:32 +00:00
SelectQueryDescription.cpp Things 2023-12-28 17:56:05 +00:00
SelectQueryDescription.h Things 2023-12-28 17:56:05 +00:00
SelectQueryInfo.cpp Fixing build. 2023-05-23 20:47:35 +00:00
SelectQueryInfo.h Support non global in mode. 2024-02-05 17:05:21 +00:00
SetSettings.cpp
SetSettings.h Move obsolete format settings to separate section 2023-09-20 16:00:28 +00:00
StatisticsDescription.cpp refine exception messages 2023-09-27 17:59:53 +02:00
StatisticsDescription.h address comments 2023-10-30 00:39:16 +01:00
StorageAzureBlob.cpp Merge branch 'master' into auto-format-detection 2024-02-13 19:08:33 +01:00
StorageAzureBlob.h Addressed comments, added test for named collection 2024-02-04 11:28:20 +01:00
StorageAzureBlobCluster.cpp Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
StorageAzureBlobCluster.h Try to detect file format automatically during schema inference if it's unknown 2024-01-23 18:59:39 +00:00
StorageBuffer.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageBuffer.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageConfiguration.h
StorageDictionary.cpp Better shutdown 2023-11-06 15:47:57 +01:00
StorageDictionary.h Better shutdown 2023-11-06 15:47:57 +01:00
StorageDistributed.cpp Merge pull request #59987 from nickitat/fix_data_race_storage_distributed 2024-02-15 02:55:08 +01:00
StorageDistributed.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageDummy.cpp Refactor a bit. 2023-06-16 19:38:50 +00:00
StorageDummy.h Revert "Revert "Planner prepare filters for analysis"" 2023-08-24 12:32:56 +02:00
StorageExecutable.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageExecutable.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageExternalDistributed.cpp remove old code of projection analysis 2023-10-12 23:25:20 +00:00
StorageExternalDistributed.h
StorageFactory.cpp
StorageFactory.h Fix StorageURL forgetting headers on server restart 2024-01-19 11:35:12 -08:00
StorageFile.cpp Merge branch 'master' into auto-format-detection 2024-02-13 19:08:33 +01:00
StorageFile.h Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
StorageFileCluster.cpp Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
StorageFileCluster.h Try to detect file format automatically during schema inference if it's unknown 2024-01-23 18:59:39 +00:00
StorageFuzzJSON.cpp Update src/Storages/StorageFuzzJSON.cpp 2023-12-19 10:17:15 -08:00
StorageFuzzJSON.h Add malformed output generation to JSON fuzzer (#57646) 2023-12-13 19:59:31 +01:00
StorageGenerateRandom.cpp Fix tests 2023-12-12 14:13:15 +00:00
StorageGenerateRandom.h
StorageInMemoryMetadata.cpp Merge branch 'master' into non-ready-set-ttl 2024-01-13 21:11:51 +01:00
StorageInMemoryMetadata.h Slightly more things 2023-12-28 17:56:05 +00:00
StorageInput.cpp Fixing style. 2023-11-14 14:55:21 +00:00
StorageInput.h Fixing style. 2023-11-15 13:56:51 +00:00
StorageJoin.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageJoin.h StorageJoin: supports trivial count() 2023-10-19 06:30:25 +00:00
StorageKeeperMap.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageKeeperMap.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageLog.cpp fix nested size0 for storage Log 2024-01-15 17:26:44 +00:00
StorageLog.h fix nested size0 for storage Log 2024-01-15 17:26:44 +00:00
StorageLogSettings.cpp
StorageLogSettings.h Add missing includes 2024-01-13 01:48:55 +03:00
StorageMaterializedMySQL.cpp
StorageMaterializedMySQL.h Style fix 2023-08-14 12:30:29 +04:00
StorageMaterializedView.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageMaterializedView.h Merge pull request #57520 from Avogar/ignore-mv-with-dropped-target-table 2024-01-04 15:33:27 +01:00
StorageMemory.cpp Support backups for compressed in-memory tables 2024-01-28 23:06:50 +01:00
StorageMemory.h allow ALTER for TEMPORARY table 2023-11-30 21:42:12 +03:00
StorageMerge.cpp Fix typo 2024-02-09 11:45:15 +01:00
StorageMerge.h WIP on calculate aliases only once 2024-02-02 13:45:04 +00:00
StorageMergeTree.cpp Merge branch 'master' into allow-parallel-replicas-for-join-with-analyzer-2 2024-02-13 11:11:03 +01:00
StorageMergeTree.h Disable mutable operations into {Replicated,}MergeTree with static storage 2024-01-26 16:47:35 +01:00
StorageMongoDB.cpp Fix build 2023-07-06 06:31:09 +00:00
StorageMongoDB.h Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageMongoDBSocketFactory.cpp
StorageMongoDBSocketFactory.h
StorageMySQL.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageMySQL.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageNull.cpp Minor code cleanup: remove some redundant includes of InterpreterAlterQuery.h 2023-08-23 14:16:36 +00:00
StorageNull.h fix: StorageNull supports subcolumns 2023-10-22 16:24:01 +08:00
StoragePostgreSQL.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StoragePostgreSQL.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageProxy.h Merge remote-tracking branch 'upstream/master' into HEAD 2024-01-03 16:01:44 +00:00
StorageRedis.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageRedis.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageReplicatedMergeTree.cpp Revert "ReplicatedMergeTree invalid metadata_version fix" 2024-02-16 11:44:16 +01:00
StorageReplicatedMergeTree.h Convert default_replica_path/default_replica_name into server settings 2024-02-12 21:05:58 +01:00
StorageS3.cpp Merge branch 'master' into auto-format-detection 2024-02-13 19:08:33 +01:00
StorageS3.h Merge branch 'master' into auto-format-detection 2024-02-13 19:08:33 +01:00
StorageS3Cluster.cpp Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
StorageS3Cluster.h Try to detect file format automatically during schema inference if it's unknown 2024-01-23 18:59:39 +00:00
StorageS3Settings.cpp Move Allocator code into module part 2023-12-27 15:42:08 +01:00
StorageS3Settings.h Minor cache changes 2023-12-04 19:02:37 +01:00
StorageSet.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageSet.h Merging #52352 2023-10-14 02:52:53 +02:00
StorageSnapshot.cpp fix build and tests 2024-01-03 16:59:13 +00:00
StorageSnapshot.h Revert "Revert "Planner prepare filters for analysis"" 2023-08-24 12:32:56 +02:00
StorageSQLite.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageSQLite.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageStripeLog.cpp Updated implementation 2024-01-25 14:31:49 +03:00
StorageStripeLog.h Updated implementation 2024-01-25 14:31:49 +03:00
StorageTableFunction.h Better shutdown 2023-11-06 15:47:57 +01:00
StorageURL.cpp Style check for abbreviations 2024-02-17 02:14:15 +01:00
StorageURL.h Try to detect file format automatically during schema inference if it's unknown 2024-01-23 18:59:39 +00:00
StorageURLCluster.cpp Fix build 2024-01-26 15:50:16 +01:00
StorageURLCluster.h Try to detect file format automatically during schema inference if it's unknown 2024-01-23 18:59:39 +00:00
StorageValues.cpp
StorageValues.h Disable parallelize_output_from_storages for storages with only one block 2023-06-14 19:11:23 +03:00
StorageView.cpp Fix inconsistent formatting of SELECT ... FROM (EXPLAIN ...) 2024-02-18 05:38:34 +01:00
StorageView.h Create new StorageView with substituted parameters for every SELECT query of a parameterized view 2023-07-25 14:04:55 +02:00
StorageXDBC.cpp Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
StorageXDBC.h Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
TableLockHolder.h
transformQueryForExternalDatabase.cpp Fix transform query for external database 2023-12-15 08:34:58 +00:00
transformQueryForExternalDatabase.h Better 2023-08-10 06:34:10 +00:00
transformQueryForExternalDatabaseAnalyzer.cpp Fixing 01086_odbc_roundtrip with analyzer. 2023-08-31 15:23:27 +00:00
transformQueryForExternalDatabaseAnalyzer.h
TTLDescription.cpp Merge branch 'master' into non-ready-set-ttl 2024-01-13 21:11:51 +01:00
TTLDescription.h Merge branch 'master' into non-ready-set-ttl 2024-01-13 21:11:51 +01:00
TTLMode.h
UVLoop.h Updated implementation 2024-01-25 14:31:49 +03:00
VirtualColumnUtils.cpp Forward declaration for PeekableReadBuffer 2024-01-30 19:24:19 +01:00
VirtualColumnUtils.h Fixing index hint 2024-01-05 11:50:09 +00:00