ClickHouse/src/Storages
Azat Khuzhin 1e2b0c8dc4 Fix possible UB in Set skipping index for functions with incorrect args
Usually functions are validated before index analysis, however it is
not the case for parallel replicas, and it requires additional checks
before interpreting function arguments.

<details>

<summary>stack trace</summary>

```
==172==WARNING: MemorySanitizer: use-of-uninitialized-value
    0 0x55cf82aedd6c in DB::ASTFunction* typeid_cast<DB::ASTFunction*, DB::IAST>(DB::IAST*) build_docker/./src/Common/typeid_cast.h:42:73
    1 0x55cf82aedd6c in DB::TypePromotion<DB::IAST>::CastHelper<DB::ASTFunction, false, false>::value(DB::IAST*) build_docker/./src/Common/TypePromotion.h:38:43
    2 0x55cf82aedd6c in std::__1::invoke_result<decltype(&CastHelper<DB::ASTFunction, false>::value), DB::TypePromotion<DB::IAST>::CastHelper<DB::ASTFunction, false, std::is_reference_v<DB::ASTFunction>>, DB::IAST*>::type DB::TypePromotion<DB::IAST>::as<DB::ASTFunction>() build_docker/./src/>
    3 0x55cf82aedd6c in DB::MergeTreeIndexConditionSet::operatorFromAST(std::__1::shared_ptr<DB::IAST>&) build_docker/./src/Storages/MergeTree/MergeTreeIndexSet.cpp:602:25
    4 0x55cf82ae5bc1 in DB::MergeTreeIndexConditionSet::traverseAST(std::__1::shared_ptr<DB::IAST>&) const build_docker/./src/Storages/MergeTree/MergeTreeIndexSet.cpp:547:9
    5 0x55cf82ae5de6 in DB::MergeTreeIndexConditionSet::traverseAST(std::__1::shared_ptr<DB::IAST>&) const build_docker/./src/Storages/MergeTree/MergeTreeIndexSet.cpp:552:13
    6 0x55cf82ae06db in DB::MergeTreeIndexConditionSet::MergeTreeIndexConditionSet() build_docker/.>
    ...
    12 0x55cf82aef09c in DB::MergeTreeIndexSet::createIndexCondition() const build_docker/./src/Storages/MergeTree/MergeTreeIndexSet.cpp:703:12
    13 0x55cf84951cd4 in DB::buildIndexes()
    14 0x55cf84955ed3 in DB::ReadFromMergeTree::selectRangesToReadImpl()
    15 0x55cf8494caef in DB::ReadFromMergeTree::selectRangesToRead()
    16 0x55cf82a409a9 in DB::MergeTreeDataSelectExecutor::estimateNumMarksToRead()
    17 0x55cf827f728d in DB::MergeTreeData::canUseParallelReplicasBasedOnPKAnalysis()
    18 0x55cf827f627e in DB::MergeTreeData::getQueryProcessingStage()
    19 0x55cf7f2f4969 in DB::InterpreterSelectQuery::getSampleBlockImpl() build_docker/./src/Interpreters/InterpreterSelectQuery.cpp:937:31
    20 0x55cf7f2daa00 in DB::InterpreterSelectQuery::InterpreterSelectQuery()
    24 0x55cf7f520b98 in DB::InterpreterSelectWithUnionQuery::buildCurrentChildInterpreter()
    25 0x55cf7f51b6cd in DB::InterpreterSelectWithUnionQuery::InterpreterSelectWithUnionQuery()
    27 0x55cf7f1d4ea9 in DB::InterpreterFactory::get(std::__1::shared_ptr<DB::IAST>&, std::__1::shared_ptr<DB::Context>, DB::SelectQueryOptions const&) build_docker/./src/Interpreters/InterpreterFactory.cpp:162:16
    28 0x55cf8012e485 in DB::executeQueryImpl(char const*, char const*, std::__1::shared_ptr<DB::Context>, bool, DB::QueryProcessingStage::Enum, DB::ReadBuffer*) build_docker/./src/Interpreters/executeQuery.cpp:1032:31
    29 0x55cf80121bc1 in DB::executeQuery(std::__1::basic_string<char, std::__1::char_traits<char>, std::__1::allocator<char>> const&, std::__1::shared_ptr<DB::Context>, bool, DB::QueryProcessingStage::Enum) build_docker/./src/Interpreters/executeQuery.cpp:1229:30
    30 0x55cf8389295f in DB::TCPHandler::runImpl() build_docker/./src/Server/TCPHandler.cpp:424:24
    31 0x55cf838d7dfb in DB::TCPHandler::run() build_docker/./src/Server/TCPHandler.cpp:2050:9
```

</details>

CI: https://s3.amazonaws.com/clickhouse-test-reports/53214/d99b10c340909ed4ee2e6edf0921e8a2f8561b0d/fuzzer_astfuzzermsan/report.html
Signed-off-by: Azat Khuzhin <a.khuzhin@semrush.com>
2023-08-18 18:02:46 +02:00
..
Cache Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
DataLakes Merge branch 'master' into formats-with-subcolumns 2023-08-02 15:24:56 +02:00
Distributed Add ability to turn off flush of Distributed on DETACH/DROP/server shutdown 2023-08-17 08:58:06 +02:00
examples Fix build 2023-05-03 00:09:52 +02:00
FileLog Highly questionable refactoring (getInputMultistream() nonsense) 2023-04-17 04:58:32 +00:00
fuzzers Ditch tons of garbage 2023-08-09 02:19:02 +02:00
HDFS Merge branch 'master' into formats-with-subcolumns 2023-08-04 13:01:12 +02:00
Hive Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
Kafka review fixes 2023-07-28 16:06:03 +00:00
LiveView Implement endianness-independent SipHash and MergeTree checksum serialization 2023-07-19 10:01:55 -07:00
MeiliSearch Make IAST::FormatSettings more regular, pt. III 2023-07-20 10:34:05 +00:00
MergeTree Fix possible UB in Set skipping index for functions with incorrect args 2023-08-18 18:02:46 +02:00
MySQL Fox 2023-04-13 19:36:25 +02:00
NATS Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
PostgreSQL Update src/Storages/PostgreSQL/MaterializedPostgreSQLConsumer.cpp 2023-07-20 19:03:06 +02:00
RabbitMQ Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
RocksDB Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
S3Queue less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
System Merge pull request #53424 from ClickHouse/exception_message_patterns6 2023-08-17 17:58:04 +03:00
tests Fix build 2023-08-10 16:39:43 +00:00
WindowView Remove non-const function Context::getClientInfo(). 2023-07-17 15:02:07 +02:00
AlterCommands.cpp less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
AlterCommands.h
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 less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
ColumnsDescription.h less exceptions with runtime format string 2023-08-14 21:45:59 +02:00
CompressionCodecSelector.h
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
ExternalDataSourceConfiguration.cpp Remove redundant 2023-02-27 12:32:13 +01:00
ExternalDataSourceConfiguration.h Finish for streaming engines 2023-02-21 14:50:55 +01: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
Freeze.h
getStructureOfRemoteTable.cpp Merge pull request #51141 from ClickHouse/azure_blob_storage_sas_token 2023-08-01 12:27:17 +02:00
getStructureOfRemoteTable.h
getVirtualsForStorage.cpp
getVirtualsForStorage.h
IMessageProducer.cpp
IMessageProducer.h
IndicesDescription.cpp Make serializeAST() more regular 2023-07-20 10:39:26 +00:00
IndicesDescription.h
IStorage_fwd.h
IStorage.cpp Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03:00
IStorage.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03:00
IStorageCluster.cpp Fix schema inference with named collection, refactor Cluster table functions 2023-05-12 13:58:45 +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 Remove superfluous includes of logger_userful.h from headers 2023-04-10 17:59:30 +02:00
LightweightDeleteDescription.cpp
LightweightDeleteDescription.h
MarkCache.h Implement endianness-independent SipHash and MergeTree checksum serialization 2023-07-19 10:01:55 -07: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 Revert "Revert "Fix several RENAME COLUMN bugs."" 2023-02-27 12:27:57 +01: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
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
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 Merge remote-tracking branch 'upstream/master' into s3queue 2023-06-30 13:56:43 +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
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 select only required columns from system.databases 2023-08-02 23:23:52 +02:00
SetSettings.cpp
SetSettings.h
StorageAzureBlob.cpp Merge branch 'master' into azure_table_function_cluster 2023-08-04 21:39:32 +02:00
StorageAzureBlob.h Merge branch 'master' into azure_table_function_cluster 2023-08-04 21:39:32 +02:00
StorageAzureBlobCluster.cpp Fixed glob iterator for table function cluster path without regex characters 2023-06-28 11:09:19 +02:00
StorageAzureBlobCluster.h Fixed cluster with distributed_processing 2023-06-09 15:17:08 +02:00
StorageBuffer.cpp Merge pull request #49723 from DimasKovas/fix_storage_buffer_reschedule 2023-08-12 03:28:23 +03:00
StorageBuffer.h Make shutdown of replicated tables softer 2023-07-05 18:11:25 +02:00
StorageConfiguration.h
StorageDictionary.cpp Apply some CTAD 2023-03-02 13:36:47 +00:00
StorageDictionary.h Propagate input_format_parquet_preserve_order to parallelizeOutputAfterReading() 2023-05-05 04:20:27 +00:00
StorageDistributed.cpp Add ability to turn off flush of Distributed on DETACH/DROP/server shutdown 2023-08-17 08:58:06 +02:00
StorageDistributed.h Merge branch 'master' into add_delay_for_replicated 2023-07-24 16:07:38 +02:00
StorageDummy.cpp Refactor a bit. 2023-06-16 19:38:50 +00:00
StorageDummy.h Fix Object data type for StorageDistributed 2023-06-02 23:41:25 +02:00
StorageExecutable.cpp
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 new redis engine schema design 2023-06-02 10:05:54 +08:00
StorageFile.cpp Revert "Revert "Upload build time-trace data to CI database"" 2023-08-09 16:11:17 +03:00
StorageFile.h Merge branch 'master' into add-reading-from-archives 2023-08-04 12:42:46 +00:00
StorageGenerateRandom.cpp Control memory usage in generateRandom 2023-06-04 04:44:51 +02: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
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 Cosmetics: Rename dropCache() to clearCaches() 2023-08-14 15:42:32 +00: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 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 Add a note about not working _table filter for Merge with analyzer 2023-07-27 16:35:17 +02:00
StorageMerge.h Fixes for parallel replicas (#50195) 2023-05-25 14:41:04 +02:00
StorageMergeTree.cpp Merge remote-tracking branch 'rschu1ze/master' into consistency-fixes-for-caches 2023-08-16 21:26:04 +00:00
StorageMergeTree.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03: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 Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02:00
StorageNull.cpp Only check MV on ALTER when necessary 2023-03-27 17:45:15 +02: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 Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03: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 exception_message_patterns6 2023-08-16 22:58:24 +02:00
StorageReplicatedMergeTree.h Revert "Improve CHECK TABLE system query" 2023-08-10 14:44:16 +03:00
StorageS3.cpp Merge branch 'master' into formats-with-subcolumns 2023-08-02 15:24:56 +02:00
StorageS3.h Merge branch 'master' into formats-with-subcolumns 2023-08-02 15:24:56 +02:00
StorageS3Cluster.cpp Merge branch 'master' into better-progress-bar-2 2023-07-24 19:59:38 +02:00
StorageS3Cluster.h Support reading subcolumns from file/s3/hdfs/url/azureBlobStorage table functions 2023-07-04 21:17:26 +00: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 Fixed clang tidy build by removing unued variable 2023-07-29 11:15:56 +02:00
StorageSnapshot.h Create new StorageView with substituted parameters for every SELECT query of a parameterized view 2023-07-25 14:04:55 +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 Merge pull request #52626 from ClickHouse/fix_deadlock_in_persistent_table_functions 2023-07-27 09:38:48 +02:00
StorageURL.cpp Merge branch 'master' into formats-with-subcolumns 2023-08-02 15:24:56 +02:00
StorageURL.h Merge branch 'master' into formats-with-subcolumns 2023-07-26 13:30:35 +02:00
StorageURLCluster.cpp Fix build 2023-06-21 02:33:38 +02:00
StorageURLCluster.h Support reading subcolumns from file/s3/hdfs/url/azureBlobStorage table functions 2023-07-04 21:17:26 +00: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 Create new StorageView with substituted parameters for every SELECT query of a parameterized view 2023-07-25 14:04:55 +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 Better 2023-08-10 06:34:10 +00:00
StorageXDBC.h Correctly disable async insert with deduplication when it's not needed (#50663) 2023-06-07 20:33:08 +02: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 Better transformQueryForExternalDatabase for analyzer 2023-03-22 16:58:34 +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
VirtualColumnUtils.cpp RFC: Fix filtering by virtual columns with OR expression 2023-07-27 16:35:17 +02:00
VirtualColumnUtils.h