ClickHouse/src/Storages
Nikolay Degterinsky 9683adfb03
Merge pull request #56830 from evillique/better-add-column
Fix flattening existing Nested columns during ADD COLUMN
2023-11-19 16:21:56 +01:00
..
Cache used assert_cast instead of dynamic_cast 2023-09-05 22:57:40 +00:00
DataLakes Allow repeated adds and removes for delta lake files 2023-11-02 13:40:50 +01:00
Distributed Add metrics for the number of queued jobs, which is useful for the IO thread pool 2023-11-18 19:07:59 +01:00
examples Fix build 2023-05-03 00:09:52 +02:00
FileLog Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +01:00
fuzzers fix fuzzers, cmake refactor, add target fuzzers 2023-09-01 14:20:50 +00:00
HDFS Update after review 2023-11-15 16:37:24 +01:00
Hive Add metrics for the number of queued jobs, which is useful for the IO thread pool 2023-11-18 19:07:59 +01:00
Kafka Fix build 2023-11-13 10:09:23 +01:00
LiveView Better shutdown 2023-11-06 15:47:57 +01:00
MergeTree Merge pull request #56958 from ClickHouse/metric-queued-jobs 2023-11-19 10:37:18 +01:00
MySQL Fox 2023-04-13 19:36:25 +02:00
NATS Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +01:00
PostgreSQL Better shutdown 2023-11-06 15:47:57 +01:00
RabbitMQ Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +01:00
RocksDB Fix tests. 2023-11-07 11:43:16 +00:00
S3Queue Fix test 2023-11-08 12:29:40 +01:00
System Add metrics for the number of queued jobs, which is useful for the IO thread pool 2023-11-18 19:07:59 +01:00
tests Fix 2023-11-09 06:23:23 +00:00
WindowView Better shutdown 2023-11-06 15:47:57 +01:00
AlterCommands.cpp Fix a bug 2023-11-18 15:44:45 +00: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 Fix style 2022-06-27 13:56:43 +00:00
CheckResults.h Improve CHECK TABLE system query 2023-10-23 09:35:26 +00:00
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 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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ConstraintsDescription.cpp Remove default value for argument 'ignore_aliases' from IAST::getTreeHash() 2023-11-13 10:27:38 +00:00
ConstraintsDescription.h Add support for substitute column 2023-03-17 13:38:01 +00:00
DataDestinationType.h
ExecutableSettings.cpp Follow-up to PR #37300: semicolon warnings 2022-05-20 15:06:34 +02:00
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 fix underscore 2022-08-23 13:59:02 +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 race 2023-02-13 16:02:41 +01:00
IMessageProducer.h Fix race 2023-02-13 16:02:41 +01:00
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 wip CHECK query for all tables 2023-10-27 15:22:10 +00:00
IStorage.h Merge pull request #56367 from canhld94/ch_table_reinit_new_disk 2023-11-14 15:54:22 +01: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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
KeyDescription.h
KVStorageUtils.cpp Remove default value for argument 'ignore_aliases' from IAST::getTreeHash() 2023-11-13 10:27:38 +00:00
KVStorageUtils.h Fix rocksdb with analyzer. 2023-11-06 18:46:39 +00:00
LightweightDeleteDescription.cpp Capitalized const name 2022-07-25 16:32:16 +02:00
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 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 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 Fix format 2023-11-09 17:25:20 +01: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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
PartitionCommands.h Support clang-18 (Wmissing-field-initializers) 2023-08-23 15:53:45 +02:00
PartitionedSink.cpp Remove C++ templates 2023-11-10 05:25:02 +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 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 Fixes for -Wshorten-64-to-32 2022-10-21 13:25:19 +02:00
ReadFinalForExternalReplicaStorage.h Fixes for -Wshorten-64-to-32 2022-10-21 13:25:19 +02:00
ReadInOrderOptimizer.cpp Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ReadInOrderOptimizer.h better support of read_in_order in case of fixed prefix of sorting key 2022-07-01 16:45:01 +00:00
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
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.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 Add metrics for the number of queued jobs, which is useful for the IO thread pool 2023-11-18 19:07:59 +01: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 Re-fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table (#56140) 2023-10-31 13:41:54 -07:00
StorageBuffer.h Re-fix 'Block structure mismatch' on concurrent ALTER and INSERTs in Buffer table (#56140) 2023-10-31 13:41:54 -07:00
StorageConfiguration.h Simplify code around storage s3 configuration 2023-02-06 16:23:17 +01:00
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 Add metrics for the number of queued jobs, which is useful for the IO thread pool 2023-11-18 19:07:59 +01:00
StorageDistributed.h Merge pull request #56367 from canhld94/ch_table_reinit_new_disk 2023-11-14 15:54:22 +01: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 Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
StorageExternalDistributed.cpp Add schema inference to more table engines 2023-05-19 00:44:27 +00: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 Refactor IHints 2023-09-10 03:48:12 +00:00
StorageFile.cpp small fixes after review 2023-11-16 00:01:36 +01: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 Fix SipHash integer hashing and byte order issue in GenerateRandom for s390x 2023-03-14 11:54:17 -04:00
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 Fixing style. 2023-11-14 14:55:21 +00:00
StorageInput.h Fixing style. 2023-11-15 13:56:51 +00:00
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 Cleanup 2023-11-17 10:27:19 +00:00
StorageKeeperMap.h Small fixes and add test 2023-11-09 15:56:57 +00:00
StorageLog.cpp wip CHECK query for all tables 2023-10-27 15:22:10 +00:00
StorageLog.h wip CHECK query for all tables 2023-10-27 15:22:10 +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 Fixes for -Wshorten-64-to-32 2022-10-21 13:25:19 +02:00
StorageMaterializedMySQL.h Style fix 2023-08-14 12:30:29 +04:00
StorageMaterializedView.cpp Merge pull request #56377 from ClickHouse/minor-improvements-for-s3-queue 2023-11-10 11:28:09 +01:00
StorageMaterializedView.h Better shutdown 2023-11-06 15:47:57 +01: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 Merge pull request #54905 from nickitat/fix_key_analysis_merge_engine 2023-11-09 14:52:54 +01:00
StorageMerge.h better 2023-09-26 11:51:34 +02:00
StorageMergeTree.cpp Merge pull request #56502 from amosbird/fix-56481 2023-11-11 02:29:04 +01:00
StorageMergeTree.h Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
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 Merge remote-tracking branch 'origin/master' into minor-improvements-for-s3-queue 2023-11-06 15:51:11 +01: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 pull request #56102 from ClickHouse/coverage 2023-11-16 23:23:40 +01:00
StorageReplicatedMergeTree.h Fix unexpected parts handling (#56693) 2023-11-15 16:54:47 +01:00
StorageS3.cpp Merge pull request #56314 from CheSema/s3-aggressive-timeouts 2023-11-19 14:12:14 +01:00
StorageS3.h get rid off of client_with_long_timeout_ptr 2023-11-14 11:34:12 +01: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 get rid off of client_with_long_timeout_ptr 2023-11-14 11:34:12 +01: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 Split source and worker processors in InterpreterCheckQuery 2023-10-27 15:22:10 +00:00
StorageStripeLog.h wip CHECK query for all tables 2023-10-27 15:22:10 +00:00
StorageTableFunction.h Better shutdown 2023-11-06 15:47:57 +01:00
StorageURL.cpp Add ClickHouse setting to disable tunneling for HTTPS requests over HTTP proxy (#55033) 2023-11-04 13:47:52 -04: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 Fixes for -Wshorten-64-to-32 2022-10-21 13:25:19 +02:00
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
transformQueryForExternalDatabase.cpp Update src/Storages/transformQueryForExternalDatabase.cpp 2023-11-08 23:20:12 +08: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 Support transformQueryForExternalDatabase for analyzer 2023-03-22 08:59:04 +01:00
TTLDescription.cpp enable_qpl_deflate_codec --> enable_deflate_qpl_codec 2023-06-09 12:43:33 +00:00
TTLDescription.h Cleanups 2022-07-24 12:21:18 +02:00
TTLMode.h
UVLoop.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
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