ClickHouse/src/Storages
Azat Khuzhin c048ab9d6d Fix processing INSERT after metadata mutations that had not been applied yet
The problem is that right now INSERT may write old columns even after
metadata mutation (i.e. RENAME), because it will simply write the new
block number that is bigger then metadata version, and so this part
won't be mutated.

To fix this, it should get the mutation version from metadata - this is
quite a major change!

Fixes: 2555_davengers_rename_chain
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2024-04-25 14:35:21 +02:00
..
Cache Fixing more headers 2024-02-29 15:40:30 +00:00
DataLakes Arrow schema to ClickHouse schema Nullable fix 2024-04-07 20:35:27 +03:00
Distributed More fixes 2024-04-03 19:18:31 +00:00
examples Fix build of examples 2024-03-10 10:03:31 +01:00
FileLog Merge pull request #62421 from ClickHouse/tavplubix-patch-13 2024-04-09 20:41:28 +00:00
fuzzers
HDFS Merge remote-tracking branch 'upstream/master' into hive-segfault 2024-04-15 12:50:47 +00:00
Hive Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
Kafka Add client.id to librdkafka logs 2024-04-20 10:42:27 +00:00
LiveView Merge remote-tracking branch 'upstream/master' into HEAD 2024-03-06 14:14:42 +00:00
MaterializedView Remove nested dependency on DateLutImpl 2024-04-02 14:45:48 +02:00
MergeTree Fix processing INSERT after metadata mutations that had not been applied yet 2024-04-25 14:35:21 +02:00
MySQL
NATS Remove a few nested include dependencies 2024-04-02 13:43:14 +02:00
PostgreSQL Limit backtracking in parser 2024-03-17 19:54:45 +01:00
RabbitMQ Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
RocksDB Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
S3Queue Remove nested dependency on DateLutImpl 2024-04-02 14:45:48 +02:00
Statistics
System Merge pull request #62152 from JackyWoo/add_query_progress_to_table_zookeeper 2024-04-20 00:48:31 +00:00
tests Revert "Merge pull request #61564 from liuneng1994/optimize_in_single_value" 2024-04-01 13:51:56 +02:00
WindowView fix 2024-04-07 20:05:41 +02:00
AlterCommands.cpp Don't access static members through instance 2024-04-03 18:50:33 +00:00
AlterCommands.h Add definers for views (Attempt 2) (#60439) 2024-02-27 19:00:17 -05:00
buildQueryTreeForShard.cpp Review fixes. 2024-04-09 12:30:07 +00:00
buildQueryTreeForShard.h Support non global in mode. 2024-02-05 17:05:21 +00:00
checkAndGetLiteralArgument.cpp Fixing NULL random seed for generateRandom with analyzer. 2024-04-03 13:12:05 +00:00
checkAndGetLiteralArgument.h
CheckResults.h
CMakeLists.txt
ColumnDefault.cpp
ColumnDefault.h
ColumnDependency.h
ColumnsDescription.cpp Limit backtracking in parser 2024-03-17 19:54:45 +01:00
ColumnsDescription.h Merge remote-tracking branch 'upstream/master' into HEAD 2024-03-06 14:14:42 +00:00
CompressionCodecSelector.h
ConstraintsDescription.cpp Limit backtracking in parser 2024-03-17 19:54:45 +01:00
ConstraintsDescription.h
DataDestinationType.h
ExecutableSettings.cpp Reduce header dependencies 2024-03-19 17:04:29 +01:00
ExecutableSettings.h Reduce header dependencies 2024-03-19 17:04:29 +01:00
ExternalDataSourceConfiguration.cpp
ExternalDataSourceConfiguration.h
extractKeyExpressionList.cpp
extractKeyExpressionList.h
extractTableFunctionArgumentsFromSelectQuery.cpp
extractTableFunctionArgumentsFromSelectQuery.h
Freeze.cpp Updated implementation 2024-01-25 14:31:49 +03:00
Freeze.h Fixing more headers 2024-02-29 15:40:30 +00:00
getStructureOfRemoteTable.cpp Reduce header dependencies 2024-03-19 17:04:29 +01: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 fix 2024-04-03 11:52:55 +00:00
IndicesDescription.h Always apply first minmax index among available skip indices 2024-03-01 19:31:15 +00:00
IStorage_fwd.h Rename "common" to "base" 2021-10-02 10:13:14 +03:00
IStorage.cpp Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
IStorage.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
IStorageCluster.cpp Fix: *Cluster table functions progress bar 2024-02-28 16:28:28 +00:00
IStorageCluster.h Merge branch 'master' into auto-format-detection 2024-01-25 22:11:07 +01:00
KeyDescription.cpp Don't access static members through instance 2024-04-03 18:50:33 +00:00
KeyDescription.h
KVStorageUtils.cpp Fix more tests 2024-02-26 02:31:59 +08:00
KVStorageUtils.h Fix more tests 2024-02-26 02:31:59 +08:00
MarkCache.cpp Avoid instantiation of CacheBase's ctor in header file 2024-03-20 17:00:36 +00:00
MarkCache.h Avoid instantiation of CacheBase's ctor in header file 2024-03-20 17:00:36 +00:00
MemorySettings.cpp small fixes 2024-04-08 17:59:54 +02:00
MemorySettings.h Support ALTER MODIFY SETTING for Memory tables 2024-03-28 19:55:22 +08:00
MessageQueueSink.cpp Updated implementation 2024-01-25 14:31:49 +03:00
MessageQueueSink.h
MutationCommands.cpp Limit backtracking in parser 2024-03-17 19:54:45 +01:00
MutationCommands.h
NamedCollectionsHelpers.cpp Revert "Revert "Add table function mergeTreeIndex"" 2024-02-26 22:47:39 +00:00
NamedCollectionsHelpers.h Fixing more headers 2024-02-29 15:40:30 +00:00
PartitionCommands.cpp add FORGET PARTITION query to remove old parition nodes from ZooKeeper 2024-02-02 10:03:31 +00:00
PartitionCommands.h add FORGET PARTITION query to remove old parition nodes from ZooKeeper 2024-02-02 10:03:31 +00:00
PartitionedSink.cpp
PartitionedSink.h
prepareReadingFromFormat.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
prepareReadingFromFormat.h better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
ProjectionsDescription.cpp Limit backtracking in parser 2024-03-17 19:54:45 +01:00
ProjectionsDescription.h
ReadFinalForExternalReplicaStorage.cpp
ReadFinalForExternalReplicaStorage.h
ReadInOrderOptimizer.cpp
ReadInOrderOptimizer.h
RedisCommon.cpp
RedisCommon.h
registerStorages.cpp
registerStorages.h
removeGroupingFunctionSpecializations.cpp
removeGroupingFunctionSpecializations.h
RenamingRestrictions.h
ReplaceAliasByExpressionVisitor.cpp
ReplaceAliasByExpressionVisitor.h
SelectQueryDescription.cpp Things 2023-12-28 17:56:05 +00:00
SelectQueryDescription.h Things 2023-12-28 17:56:05 +00:00
SelectQueryInfo.cpp
SelectQueryInfo.h Fix most tests 2024-02-26 02:31:58 +08:00
SetSettings.cpp
SetSettings.h
StatisticsDescription.cpp
StatisticsDescription.h
StorageAzureBlob.cpp Merge pull request #62425 from Avogar/fix-partitioned-write 2024-04-15 10:29:18 +00:00
StorageAzureBlob.h cancel tasks on exception 2024-04-04 22:32:57 +02:00
StorageAzureBlobCluster.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
StorageAzureBlobCluster.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageBuffer.cpp fix 2024-04-08 16:13:46 +02:00
StorageBuffer.h Better if only 1 layer 2024-02-18 13:19:35 +01:00
StorageConfiguration.h
StorageDictionary.cpp Don't access static members through instance, pt. II 2024-04-07 11:09:35 +00:00
StorageDictionary.h Get rid of modifying a shared AbstractConfiguration (between StorageDictionary and ExternalDictionariesLoader). 2024-03-18 14:43:38 +01:00
StorageDistributed.cpp cancel tasks on exception 2024-04-04 22:32:57 +02:00
StorageDistributed.h Do not try to INSERT into readonly replicas for Distributed engine 2024-03-26 11:21:38 +01:00
StorageDummy.cpp Fix more tests 2024-02-26 02:31:57 +08:00
StorageDummy.h Fix filter pushdown from additional_table_filters in Merge engine in analyzer 2024-04-08 12:59:20 +00: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
StorageExternalDistributed.h
StorageFactory.cpp skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageFactory.h skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageFile.cpp Better 2024-04-09 10:40:40 +00:00
StorageFile.h Don't access static members through instance 2024-04-03 18:50:33 +00:00
StorageFileCluster.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
StorageFileCluster.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageFuzzJSON.cpp Don't access static members through instance, pt. II 2024-04-07 11:09:35 +00:00
StorageFuzzJSON.h
StorageGenerateRandom.cpp
StorageGenerateRandom.h
StorageInMemoryMetadata.cpp Fix ALTER QUERY MODIFY SQL SECURITY (#61480) 2024-03-25 20:03:02 -04:00
StorageInMemoryMetadata.h Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
StorageInput.cpp
StorageInput.h
StorageJoin.cpp Remove in-memory data parts, step 2 2024-03-10 06:44:26 +01:00
StorageJoin.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageKeeperMap.cpp Merge remote-tracking branch 'upstream/master' into HEAD 2024-03-06 14:14:42 +00:00
StorageKeeperMap.h Merge remote-tracking branch 'upstream/master' into HEAD 2024-03-06 14:14:42 +00:00
StorageLog.cpp Remove more nested includes 2024-04-03 19:06:36 +02:00
StorageLog.h skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageLogSettings.cpp
StorageLogSettings.h Add missing includes 2024-01-13 01:48:55 +03:00
StorageMaterializedMySQL.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
StorageMaterializedMySQL.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageMaterializedView.cpp Fix style 2024-04-10 21:59:14 +05:00
StorageMaterializedView.h refactoring of virtual columns 2024-02-29 18:01:54 +00:00
StorageMemory.cpp small fixes 2024-04-08 17:59:54 +02:00
StorageMemory.h Support ALTER MODIFY SETTING for Memory tables 2024-03-28 19:55:22 +08:00
StorageMerge.cpp Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageMerge.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageMergeTree.cpp Reduce overhead of the mutations for SELECTs (v2) 2024-04-25 14:35:21 +02:00
StorageMergeTree.h Reduce overhead of the mutations for SELECTs (v2) 2024-04-25 14:35:21 +02:00
StorageMergeTreeIndex.cpp fix crash in function mergeTreeIndex 2024-04-18 14:46:58 +00:00
StorageMergeTreeIndex.h Refactor more system storages. 2024-03-04 17:48:47 +00:00
StorageMongoDB.cpp
StorageMongoDB.h
StorageMongoDBSocketFactory.cpp
StorageMongoDBSocketFactory.h Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
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 Reduce header dependencies 2024-03-19 17:04:29 +01:00
StorageNull.h
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 refactoring of virtual columns 2024-02-28 17:05:00 +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 Merge pull request #62067 from nickitat/stronger_optimize_final_with_rmt 2024-04-22 19:54:48 +00:00
StorageReplicatedMergeTree.h Mark Replicated table as readonly in case of static disk (read-only/write-once) 2024-03-26 11:19:10 +01:00
StorageS3.cpp set correct defaults for s3 storage and backups 2024-04-15 13:03:55 +02:00
StorageS3.h cancel tasks on exception 2024-04-04 22:32:57 +02:00
StorageS3Cluster.cpp Merge remote-tracking branch 'upstream/master' into HEAD 2024-03-06 14:14:42 +00:00
StorageS3Cluster.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageS3Settings.cpp User specific S3 endpoint backup/restore ON CLUSTER 2024-04-03 08:55:56 +02:00
StorageS3Settings.h User specific S3 endpoint backup/restore ON CLUSTER 2024-04-03 08:55:56 +02:00
StorageSet.cpp Fix style 2024-03-28 15:22:26 +01:00
StorageSet.h
StorageSnapshot.cpp Merge remote-tracking branch 'origin/master' into analyzer-refactor-constant-name 2024-03-07 14:18:44 +01:00
StorageSnapshot.h Merge remote-tracking branch 'origin/master' into analyzer-refactor-constant-name 2024-03-07 14:18:44 +01:00
StorageSQLite.cpp skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageSQLite.h Fix INSERT into SQLite with single quote 2024-02-15 12:47:29 +01:00
StorageStripeLog.cpp skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageStripeLog.h skip sanity checks on secondary create query 2024-02-20 21:59:28 +01:00
StorageTableFunction.h
StorageURL.cpp cancel tasks on exception 2024-04-04 22:32:57 +02:00
StorageURL.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageURLCluster.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
StorageURLCluster.h Unify lightweight mutation control 2024-04-02 11:52:14 +02:00
StorageValues.cpp Update StorageValues.cpp 2024-03-18 20:13:52 +01:00
StorageValues.h Fixing 01083_expressions_in_engine_arguments with analyzer. 2024-03-18 18:53:01 +00:00
StorageView.cpp Enable clang-tidy in headers 2024-03-18 08:00:09 +00:00
StorageView.h Enable clang-tidy in headers 2024-03-18 08:00:09 +00: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 Revert "Revert "Check stack size in Parser"" 2024-02-27 01:34:50 +03:00
transformQueryForExternalDatabase.h
transformQueryForExternalDatabaseAnalyzer.cpp
transformQueryForExternalDatabaseAnalyzer.h
TTLDescription.cpp More fixes 2024-04-03 19:18:31 +00: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
VirtualColumnsDescription.cpp better interfaces for virtual columns 2024-03-01 22:29:56 +00:00
VirtualColumnsDescription.h address review comments 2024-03-05 16:03:02 +00:00
VirtualColumnUtils.cpp Review fixes. 2024-03-21 13:17:15 +00:00
VirtualColumnUtils.h Review fixes. 2024-03-21 13:17:15 +00:00