ClickHouse/src/Storages
alesapin 0d68a52c06
Merge pull request #50181 from ClickHouse/repro_49887
Make exception about `ALTER TABLE ... DROP COLUMN|INDEX|PROJECTION` more clear
2023-10-25 15:57:17 +02:00
..
Cache used assert_cast instead of dynamic_cast 2023-09-05 22:57:40 +00:00
DataLakes Add tests 2023-10-17 18:10:47 +00:00
Distributed Fix missing thread accounting for insert_distributed_sync=1 2023-10-09 15:41:52 +02:00
examples Fix build 2023-05-03 00:09:52 +02:00
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 #50181 from ClickHouse/repro_49887 2023-10-25 15:57:17 +02: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 rocksdb: not creating block during writing 2023-10-18 09:43:54 +00:00
S3Queue Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
System Support asynchronous inserts for native protocol (#54730) 2023-10-20 18:39:48 +02: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 Use query tree to rewrite the query 2023-06-12 16:51:40 +00:00
checkAndGetLiteralArgument.cpp add more check + line break 2023-07-05 15:04:38 +00:00
checkAndGetLiteralArgument.h
CheckResults.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03: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 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 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 Fix race 2023-02-13 16:02:41 +01:00
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 Support orc filter push down (file + stripe + rowgroup level) (#55330) 2023-10-24 12:08:17 -07:00
IStorage.h Implement feature 2023-10-13 16:22:18 +02:00
IStorageCluster.cpp Better 2023-09-15 11:11:04 +00:00
IStorageCluster.h Fix schema inference with named collection, refactor Cluster table functions 2023-05-12 13:58:45 +00:00
KeyDescription.cpp
KeyDescription.h
KVStorageUtils.cpp Cleanup. 2023-06-22 14:23:04 +00:00
KVStorageUtils.h
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h Capitalized const name 2022-07-25 16:32:16 +02:00
MarkCache.h Better parameter name 2023-08-22 15:43:13 +00:00
MemorySettings.cpp
MemorySettings.h
MessageQueueSink.cpp
MessageQueueSink.h release buffers with exception context 2023-06-22 13:00:13 +02:00
MutationCommands.cpp refactoring near alter conversions 2023-05-25 22:54:54 +00:00
MutationCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
NamedCollectionsHelpers.cpp Rename 2023-06-15 12:35:22 +02:00
NamedCollectionsHelpers.h Merge remote-tracking branch 'upstream/master' into add-separate-access-for-use-named-collections 2023-06-14 13:33:56 +02:00
PartitionCommands.cpp
PartitionCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
PartitionedSink.cpp release buffers with exception context 2023-06-22 13:00:13 +02: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 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 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 Remove Meilisearch 2023-09-30 06:02:38 +02:00
registerStorages.h Make code slightly better 2020-11-11 09:08:53 +08:00
removeGroupingFunctionSpecializations.cpp
removeGroupingFunctionSpecializations.h
RenamingRestrictions.h
SelectQueryDescription.cpp
SelectQueryDescription.h
SelectQueryInfo.cpp Fixing build. 2023-05-23 20:47:35 +00:00
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 Minor code cleanup: remove some redundant includes of InterpreterAlterQuery.h 2023-08-23 14:16:36 +00:00
StorageBuffer.h Make shutdown of replicated tables softer 2023-07-05 18:11:25 +02: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 SSH keys authentication (#41109) 2023-09-26 17:50:19 +02:00
StorageDistributed.h Style check 2023-09-09 12:51:34 +00: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 Refactor 2023-08-18 15:38:46 +08:00
StorageExecutable.h
StorageExternalDistributed.cpp Add schema inference to more table engines 2023-05-19 00:44:27 +00:00
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 select required columns when getting join 2023-08-07 03:15:20 +00:00
StorageKeeperMap.cpp less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
StorageKeeperMap.h Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageLog.cpp Added a new column _block_number (#47532) 2023-09-20 11:31:12 +02:00
StorageLog.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03: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 Merge branch 'master' into refactor-subqueries-for-in 2023-06-09 20:04:27 +02:00
StorageMerge.cpp Better exception messages 2023-10-08 22:54:33 +02:00
StorageMerge.h Fixes for parallel replicas (#50195) 2023-05-25 14:41:04 +02:00
StorageMergeTree.cpp Merge pull request #50181 from ClickHouse/repro_49887 2023-10-25 15:57:17 +02:00
StorageMergeTree.h Proper fix 2023-10-11 17:42:44 +02: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 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 Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageProxy.h Implement feature 2023-10-13 16:22:18 +02:00
StorageRedis.cpp add notifications in docs 2023-06-13 09:33:38 +08:00
StorageRedis.h fix build error 2023-06-12 10:15:32 +08:00
StorageReplicatedMergeTree.cpp Merge branch 'master' into repro_49887 2023-10-20 13:04:00 +02:00
StorageReplicatedMergeTree.h Proper fix 2023-10-11 17:42: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 Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageStripeLog.cpp Cosmetics: Rename dropCache() to clearCaches() 2023-08-14 15:42:32 +00:00
StorageStripeLog.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03: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 Disable parallelize_output_from_storages for storages with only one block 2023-06-14 19:11:23 +03:00
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 change alter_lock to std::timed_mutex 2021-10-26 13:37:00 +03:00
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 enable_qpl_deflate_codec --> enable_deflate_qpl_codec 2023-06-09 12:43:33 +00:00
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