ClickHouse/src/Storages
Azat Khuzhin b107712e0c Fix filtering by virtual columns with OR filter in query
The problem with the initial implementation #52653 was:
- OR can have multiple arguments
- It simply not correct to assume that if there are two arguments this is OK.
  Consider the following example:

    "WHERE (column_not_from_partition_by = 1) OR false OR false"

  Will be converted to:

    "WHERE false OR false"

And it will simply read nothing.

Yes, we could apply some optimization for bool, but this will not always
work, since to optimize things like "0 = 1" we need to execute it.

And the only way to make handle this correctly (with ability to ignore
some commands during filtering) is to make is_constant() function return
has it use something from the input block, so that we can be sure, that
we have some sensible, and not just "false".

Plus we cannot simply ignore the difference of the input and output
arguments of handling OR, we need to add always-true (1/true) if the
size is different, since otherwise it could break invariants (see
comment in the code).

This includes (but not limited to):
- _part* filtering for MergeTree
- _path/_file for various File/HDFS/... engines
- _table for Merge
- ...

P.S. analyzer does not have this bug, since it execute expression as
whole, and this is what filterBlockWithQuery() should do actually
instead, but this will be a more complex patch.

Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-10-10 20:47:52 +02:00
..
Cache used assert_cast instead of dynamic_cast 2023-09-05 22:57:40 +00:00
DataLakes Re-enable clang-tidy checks 2023-09-26 09:34:12 +00:00
Distributed rename new method 2023-08-28 16:01:00 +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 Merge pull request #54803 from Avogar/ephemeral-columns-from-files 2023-09-22 23:24:42 +02:00
Hive Move obsolete format settings to separate section 2023-09-20 16:00:28 +00: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 #55210 from HarryLeeIBM/hlee-fts-query 2023-10-04 10:01:35 +02:00
MySQL Fox 2023-04-13 19:36:25 +02:00
NATS Revert "Fix NATS high cpu usage" 2023-09-26 03:36:14 +02:00
PostgreSQL Update MaterializedPostgreSQLConsumer.cpp 2023-10-07 14:28:52 +02:00
RabbitMQ Merge pull request #54855 from Avogar/obsolete-format-settings 2023-09-25 18:00:31 +02:00
RocksDB Update StorageEmbeddedRocksDB.cpp 2023-08-17 15:14:14 +08:00
S3Queue Merge pull request #54855 from Avogar/obsolete-format-settings 2023-09-25 18:00:31 +02:00
System Merge remote-tracking branch 'rschu1ze/master' into more-mysql-compat 2023-10-08 07:49:37 +00: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 Merge pull request #54943 from amosbird/fix-54941 2023-09-23 22:11:52 +02:00
AlterCommands.h Fix tests with database replicated 2023-09-06 14:17:46 +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 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 Forbid special columns for file/s3/url/... storages, fix insert into ephemeral columns from files 2023-09-20 16:25:55 +00:00
ColumnsDescription.h Forbid special columns for file/s3/url/... storages, fix insert into ephemeral columns from files 2023-09-20 16:25:55 +00:00
CompressionCodecSelector.h Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
ConstraintsDescription.cpp Make serializeAST() more regular 2023-07-20 10:39:26 +00:00
ConstraintsDescription.h Add support for substitute column 2023-03-17 13:38:01 +00:00
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 Better naming 2020-05-28 16:09:03 +03:00
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 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 Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03:00
IStorage.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
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 Cleanup. 2023-06-22 14:23:04 +00:00
KVStorageUtils.h Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
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 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 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 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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
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
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
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 Remove unused parallel replicas coordinator in query info 2023-08-27 22:12:39 +00:00
SetSettings.cpp
SetSettings.h Move obsolete format settings to separate section 2023-09-20 16:00:28 +00:00
StorageAzureBlob.cpp Forbid special columns for file/s3/url/... storages, fix insert into ephemeral columns from files 2023-09-20 16:25:55 +00: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 Simplify code around storage s3 configuration 2023-02-06 16:23:17 +01:00
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 "Planner prepare filters for analysis" 2023-08-24 12:52:03 +03: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 Merge pull request #54803 from Avogar/ephemeral-columns-from-files 2023-09-22 23:24:42 +02: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 Better formatting for exception messages (#45449) 2023-01-24 00:13:58 +03:00
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 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
StorageMaterializedMySQL.h Style fix 2023-08-14 12:30:29 +04:00
StorageMaterializedView.cpp the worst fix for a deadlock ever 2023-08-17 00:42:51 +02:00
StorageMaterializedView.h Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02: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 More clang-tidy fixes 2023-09-21 14:40:57 +00:00
StorageMerge.h Fixes for parallel replicas (#50195) 2023-05-25 14:41:04 +02:00
StorageMergeTree.cpp Remove the removal of broken detached parts 2023-09-30 01:47:44 +02:00
StorageMergeTree.h Change interface 2023-08-29 16:26:48 +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 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 Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02: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 Change interface 2023-08-29 16:26:48 +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 SSH keys authentication (#41109) 2023-09-26 17:50:19 +02:00
StorageReplicatedMergeTree.h Make function slightly more readable 2023-09-12 21:41:05 +02:00
StorageS3.cpp Re-enable clang-tidy checks 2023-09-26 09:34:12 +00:00
StorageS3.h Resolve PR issues 2023-09-20 19:43:02 -04:00
StorageS3Cluster.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
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 Cleanup. 2023-06-22 14:23:04 +00:00
StorageSet.h Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageSnapshot.cpp Added a new column _block_number (#47532) 2023-09-20 11:31:12 +02:00
StorageSnapshot.h Revert "Planner prepare filters for analysis" 2023-08-24 12:52:03 +03: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 Forbid special columns for file/s3/url/... storages, fix insert into ephemeral columns from files 2023-09-20 16:25:55 +00:00
StorageURL.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
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 Support SAMPLE BY for VIEW 2023-09-10 08:24:46 +02: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 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
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-10 20:47:52 +02:00
VirtualColumnUtils.h Fixing 01748_partition_id_pruning 2023-09-01 14:29:31 +00:00