ClickHouse/src/Storages
Alexander Tokmakov 4d437ca462
Merge pull request #56137 from ClickHouse/fix_unexpected_parts_handling
Do not try to activate covered parts when handilng unexpected parts
2023-10-31 14:00:47 +01:00
..
Cache used assert_cast instead of dynamic_cast 2023-09-05 22:57:40 +00:00
DataLakes Rename date_time_overflow_mode -> date_time_overflow_behavior, moved it to format settings 2023-10-25 23:11:13 +02:00
Distributed Fix missing thread accounting for insert_distributed_sync=1 2023-10-09 15:41:52 +02:00
examples
FileLog Move obsolete format settings to separate section 2023-09-20 16:00:28 +00:00
fuzzers fix fuzzers, cmake refactor, add target fuzzers 2023-09-01 14:20:50 +00:00
HDFS Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
Hive Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
Kafka Merge pull request #54855 from Avogar/obsolete-format-settings 2023-09-25 18:00:31 +02:00
LiveView Merge pull request #54009 from ClickHouse/change-istorage-interface 2023-08-29 22:22:53 +02:00
MergeTree Merge pull request #56137 from ClickHouse/fix_unexpected_parts_handling 2023-10-31 14:00:47 +01:00
MySQL
NATS Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
PostgreSQL Merge branch 'master' into materialized-pg-allow-unique-identifier 2023-10-13 19:14:50 +02:00
RabbitMQ Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
RocksDB Merge pull request #55806 from canhld94/rocksdb_trivial_count 2023-10-26 11:22:38 +02:00
S3Queue Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
System Merge pull request #54947 from amosbird/minmax-combinator 2023-10-30 03:35:28 +01:00
tests Fix build 2023-08-10 16:39:43 +00:00
WindowView Merge pull request #54009 from ClickHouse/change-istorage-interface 2023-08-29 22:22:53 +02:00
AlterCommands.cpp Proper fix 2023-10-11 17:42:44 +02:00
AlterCommands.h Initial implementation 2023-10-10 19:59:48 +02: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 Merge branch 'master' into feature/mergetree-checksum-big-endian-support 2023-08-02 11:36:43 -04:00
buildQueryTreeForShard.h
checkAndGetLiteralArgument.cpp
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
ColumnsDescription.cpp Merge branch 'master' into revert-53782-revert-52762-planner-prepare-filters-for-analysis 2023-10-09 14:44:00 +02:00
ColumnsDescription.h Merge branch 'master' into revert-53782-revert-52762-planner-prepare-filters-for-analysis 2023-10-09 14:44:00 +02:00
CompressionCodecSelector.h
ConstraintsDescription.cpp Make serializeAST() more regular 2023-07-20 10:39:26 +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
extractTableFunctionArgumentsFromSelectQuery.h
Freeze.cpp Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
Freeze.h
getStructureOfRemoteTable.cpp Handle clusterAllReplicas/remote cases to avoid unnecessary logging 2023-09-12 12:52:29 +00:00
getStructureOfRemoteTable.h
IMessageProducer.cpp
IMessageProducer.h
IndicesDescription.cpp Make serializeAST() more regular 2023-07-20 10:39:26 +00:00
IndicesDescription.h Refactor IHints 2023-09-10 03:48:12 +00:00
IStorage_fwd.h
IStorage.cpp Merge pull request #53404 from ClickHouse/vdimir/check_table_improvements2 2023-10-26 10:57:44 +02:00
IStorage.h IStorage::checkDataNext returns optional 2023-10-23 10:12:30 +00:00
IStorageCluster.cpp Better 2023-09-15 11:11:04 +00:00
IStorageCluster.h
KeyDescription.cpp
KeyDescription.h
KVStorageUtils.cpp
KVStorageUtils.h
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h Better parameter name 2023-08-22 15:43:13 +00:00
MemorySettings.cpp
MemorySettings.h
MessageQueueSink.cpp
MessageQueueSink.h
MutationCommands.cpp
MutationCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
NamedCollectionsHelpers.cpp
NamedCollectionsHelpers.h
PartitionCommands.cpp
PartitionCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
PartitionedSink.cpp
PartitionedSink.h
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 less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
ProjectionsDescription.h Refactor IHints 2023-09-10 03:48:12 +00:00
ReadFinalForExternalReplicaStorage.cpp
ReadFinalForExternalReplicaStorage.h
ReadInOrderOptimizer.cpp
ReadInOrderOptimizer.h
RedisCommon.cpp
RedisCommon.h
registerStorages.cpp Remove Meilisearch 2023-09-30 06:02:38 +02:00
registerStorages.h
removeGroupingFunctionSpecializations.cpp
removeGroupingFunctionSpecializations.h
RenamingRestrictions.h
SelectQueryDescription.cpp
SelectQueryDescription.h
SelectQueryInfo.cpp
SelectQueryInfo.h Merge remote-tracking branch 'blessed/master' into parallel_replicas_row_estimation 2023-08-29 11:17:34 +02:00
SetSettings.cpp
SetSettings.h Move obsolete format settings to separate section 2023-09-20 16:00:28 +00:00
StorageAzureBlob.cpp Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageAzureBlob.h Merge pull request #54642 from rschu1ze/broken-re2st 2023-09-17 15:30:57 +02:00
StorageAzureBlobCluster.cpp Use filter by file/path before reading in url/file/hdfs table functions, reduce code duplication 2023-08-17 16:54:43 +00:00
StorageAzureBlobCluster.h Merge branch 'master' into fast-count-from-files 2023-08-23 15:13:20 +02:00
StorageBuffer.cpp Revert "Fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table (#55995)" 2023-10-29 04:34:04 +03:00
StorageBuffer.h Revert "Fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table (#55995)" 2023-10-29 04:34:04 +03:00
StorageConfiguration.h
StorageDictionary.cpp Better 2023-09-15 11:11:04 +00:00
StorageDictionary.h Better 2023-09-15 11:11:04 +00:00
StorageDistributed.cpp Fixing test. 2023-10-30 16:29:22 +00:00
StorageDistributed.h Fixing build 2023-10-04 09:01:29 +00:00
StorageDummy.cpp
StorageDummy.h Revert "Revert "Planner prepare filters for analysis"" 2023-08-24 12:32:56 +02:00
StorageExecutable.cpp Refactor 2023-08-18 15:38:46 +08:00
StorageExecutable.h
StorageExternalDistributed.cpp
StorageExternalDistributed.h
StorageFactory.cpp
StorageFactory.h Refactor IHints 2023-09-10 03:48:12 +00:00
StorageFile.cpp Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageFile.h Fix parsing error in WithNames formats while reading subset of columns with disabled input_format_with_names_use_header 2023-09-11 14:55:37 +00:00
StorageGenerateRandom.cpp Fix UInt256 and IPv4 random data generation on s390x 2023-09-13 04:55:02 -07:00
StorageGenerateRandom.h
StorageInMemoryMetadata.cpp fix projection recalc 2023-08-08 14:06:42 +00:00
StorageInMemoryMetadata.h fix recalculation of skip indexes and projcetion in ALTER DELETE 2023-07-24 14:14:12 +00:00
StorageInput.cpp
StorageInput.h
StorageJoin.cpp select required columns when getting join 2023-08-07 03:15:20 +00:00
StorageJoin.h StorageJoin: supports trivial count() 2023-10-19 06:30:25 +00:00
StorageKeeperMap.cpp less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
StorageKeeperMap.h
StorageLog.cpp Support CHECK TABLE ... PART ... query 2023-10-23 12:33:35 +00:00
StorageLog.h IStorage::checkDataNext returns optional 2023-10-23 10:12:30 +00:00
StorageLogSettings.cpp
StorageLogSettings.h
StorageMaterializedMySQL.cpp
StorageMaterializedMySQL.h Style fix 2023-08-14 12:30:29 +04:00
StorageMaterializedView.cpp Fix StorageMaterializedView::isRemote 2023-10-16 15:18:47 +00:00
StorageMaterializedView.h Fix StorageMaterializedView::isRemote 2023-10-16 15:18:47 +00:00
StorageMemory.cpp Disable using fs cache for backup/restore #2. 2023-08-03 13:52:35 +02:00
StorageMemory.h
StorageMerge.cpp Better exception messages 2023-10-08 22:54:33 +02:00
StorageMerge.h
StorageMergeTree.cpp Merge pull request #53404 from ClickHouse/vdimir/check_table_improvements2 2023-10-26 10:57:44 +02:00
StorageMergeTree.h Merge pull request #53404 from ClickHouse/vdimir/check_table_improvements2 2023-10-26 10:57:44 +02:00
StorageMongoDB.cpp Fix build 2023-07-06 06:31:09 +00:00
StorageMongoDB.h
StorageMongoDBSocketFactory.cpp
StorageMongoDBSocketFactory.h
StorageMySQL.cpp Better 2023-08-10 06:34:10 +00:00
StorageMySQL.h update comment 2023-09-19 08:18:37 +00: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 Update StoragePostgreSQL.cpp 2023-08-12 03:33:58 +03:00
StoragePostgreSQL.h
StorageProxy.h IStorage::checkDataNext returns optional 2023-10-23 10:12:30 +00:00
StorageRedis.cpp
StorageRedis.h
StorageReplicatedMergeTree.cpp do not try to activate covered parts 2023-10-30 20:17:10 +01:00
StorageReplicatedMergeTree.h Merge pull request #53404 from ClickHouse/vdimir/check_table_improvements2 2023-10-26 10:57:44 +02:00
StorageS3.cpp Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageS3.h Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageS3Cluster.cpp Fxi 2023-09-28 16:25:04 +02:00
StorageS3Cluster.h Merge branch 'master' into fast-count-from-files 2023-08-23 15:13:20 +02:00
StorageS3Settings.cpp More generic approach to disable native copy 2023-07-09 08:20:02 +02:00
StorageS3Settings.h Merge branch 'master' of github.com:ClickHouse/ClickHouse into ADQM-984 2023-07-26 09:58:26 +00:00
StorageSet.cpp Fix totalBytes() 2023-10-15 10:40:37 +02:00
StorageSet.h Merging #52352 2023-10-14 02:52:53 +02:00
StorageSnapshot.cpp Merge branch 'master' into revert-53782-revert-52762-planner-prepare-filters-for-analysis 2023-10-09 14:44:00 +02:00
StorageSnapshot.h Revert "Revert "Planner prepare filters for analysis"" 2023-08-24 12:32:56 +02:00
StorageSQLite.cpp Better 2023-08-10 06:34:10 +00:00
StorageSQLite.h
StorageStripeLog.cpp IStorage::checkDataNext returns optional 2023-10-23 10:12:30 +00:00
StorageStripeLog.h IStorage::checkDataNext returns optional 2023-10-23 10:12:30 +00:00
StorageTableFunction.h Change interface 2023-08-29 16:26:48 +02:00
StorageURL.cpp Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageURL.h Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
StorageURLCluster.cpp Use filter by file/path before reading in url/file/hdfs table functions, reduce code duplication 2023-08-17 16:54:43 +00:00
StorageURLCluster.h Merge branch 'master' into fast-count-from-files 2023-08-23 15:13:20 +02:00
StorageValues.cpp
StorageValues.h
StorageView.cpp Revert "Support SAMPLE BY for VIEW" 2023-10-09 00:08:46 +03: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 Fix parsing error in WithNames formats while reading subset of columns with disabled input_format_with_names_use_header 2023-09-11 14:55:37 +00:00
StorageXDBC.h Fix parsing error in WithNames formats while reading subset of columns with disabled input_format_with_names_use_header 2023-09-11 14:55:37 +00:00
TableLockHolder.h
transformQueryForExternalDatabase.cpp Better 2023-08-10 06:34:10 +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
TTLDescription.h
TTLMode.h
UVLoop.h
VirtualColumnUtils.cpp Fix filtering by virtual columns with OR filter in query 2023-10-16 15:42:10 +02:00
VirtualColumnUtils.h Fixing 01748_partition_id_pruning 2023-09-01 14:29:31 +00:00