From 0ec292a65f190f69c24420d5ca85d5658bffba0a Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 00:32:01 -0400 Subject: [PATCH 1/2] Revert "Revert "FuzzQuery table function"" This reverts commit ff44b206 --- .../table-functions/fuzzQuery.md | 36 ++++ programs/client/Client.h | 5 +- src/{Client => Common}/QueryFuzzer.cpp | 50 ++++-- src/{Client => Common}/QueryFuzzer.h | 35 ++-- src/Storages/StorageFuzzQuery.cpp | 169 ++++++++++++++++++ src/Storages/StorageFuzzQuery.h | 88 +++++++++ src/Storages/registerStorages.cpp | 2 + src/TableFunctions/TableFunctionFuzzQuery.cpp | 54 ++++++ src/TableFunctions/TableFunctionFuzzQuery.h | 42 +++++ src/TableFunctions/registerTableFunctions.cpp | 1 + src/TableFunctions/registerTableFunctions.h | 1 + .../03031_table_function_fuzzquery.reference | 2 + .../03031_table_function_fuzzquery.sql | 18 ++ 13 files changed, 473 insertions(+), 30 deletions(-) create mode 100644 docs/en/sql-reference/table-functions/fuzzQuery.md rename src/{Client => Common}/QueryFuzzer.cpp (97%) rename src/{Client => Common}/QueryFuzzer.h (91%) create mode 100644 src/Storages/StorageFuzzQuery.cpp create mode 100644 src/Storages/StorageFuzzQuery.h create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.cpp create mode 100644 src/TableFunctions/TableFunctionFuzzQuery.h create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.reference create mode 100644 tests/queries/0_stateless/03031_table_function_fuzzquery.sql diff --git a/docs/en/sql-reference/table-functions/fuzzQuery.md b/docs/en/sql-reference/table-functions/fuzzQuery.md new file mode 100644 index 00000000000..e15f8a40156 --- /dev/null +++ b/docs/en/sql-reference/table-functions/fuzzQuery.md @@ -0,0 +1,36 @@ +--- +slug: /en/sql-reference/table-functions/fuzzQuery +sidebar_position: 75 +sidebar_label: fuzzQuery +--- + +# fuzzQuery + +Perturbs the given query string with random variations. + +``` sql +fuzzQuery(query[, max_query_length[, random_seed]]) +``` + +**Arguments** + +- `query` (String) - The source query to perform the fuzzing on. +- `max_query_length` (UInt64) - A maximum length the query can get during the fuzzing process. +- `random_seed` (UInt64) - A random seed for producing stable results. + +**Returned Value** + +A table object with a single column containing perturbed query strings. + +## Usage Example + +``` sql +SELECT * FROM fuzzQuery('SELECT materialize(\'a\' AS key) GROUP BY key') LIMIT 2; +``` + +``` + ┌─query──────────────────────────────────────────────────────────┐ +1. │ SELECT 'a' AS key GROUP BY key │ +2. │ EXPLAIN PIPELINE compact = true SELECT 'a' AS key GROUP BY key │ + └────────────────────────────────────────────────────────────────┘ +``` diff --git a/programs/client/Client.h b/programs/client/Client.h index 7fdf77031ab..07a8e293b1a 100644 --- a/programs/client/Client.h +++ b/programs/client/Client.h @@ -11,7 +11,10 @@ class Client : public ClientApplicationBase public: using Arguments = ClientApplicationBase::Arguments; - Client() = default; + Client() + { + fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr); + } void initialize(Poco::Util::Application & self) override; diff --git a/src/Client/QueryFuzzer.cpp b/src/Common/QueryFuzzer.cpp similarity index 97% rename from src/Client/QueryFuzzer.cpp rename to src/Common/QueryFuzzer.cpp index f5b700ea529..161c38f20e0 100644 --- a/src/Client/QueryFuzzer.cpp +++ b/src/Common/QueryFuzzer.cpp @@ -68,22 +68,21 @@ Field QueryFuzzer::getRandomField(int type) { case 0: { - return bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) - / sizeof(*bad_int64_values))]; + return bad_int64_values[fuzz_rand() % std::size(bad_int64_values)]; } case 1: { static constexpr double values[] = {NAN, INFINITY, -INFINITY, 0., -0., 0.0001, 0.5, 0.9999, 1., 1.0001, 2., 10.0001, 100.0001, 1000.0001, 1e10, 1e20, - FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % (sizeof(values) / sizeof(*values))]; + FLT_MIN, FLT_MIN + FLT_EPSILON, FLT_MAX, FLT_MAX + FLT_EPSILON}; return values[fuzz_rand() % std::size(values)]; } case 2: { static constexpr UInt64 scales[] = {0, 1, 2, 10}; return DecimalField( - bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))], - static_cast(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))]) + bad_int64_values[fuzz_rand() % std::size(bad_int64_values)], + static_cast(scales[fuzz_rand() % std::size(scales)]) ); } default: @@ -165,7 +164,8 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -174,12 +174,14 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -197,7 +199,9 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.erase(arr.begin() + pos); - std::cerr << "erased\n"; + + if (debug_stream) + *debug_stream << "erased\n"; } if (fuzz_rand() % 5 == 0) @@ -206,12 +210,16 @@ Field QueryFuzzer::fuzzField(Field field) { size_t pos = fuzz_rand() % arr.size(); arr.insert(arr.begin() + pos, fuzzField(arr[pos])); - std::cerr << fmt::format("inserted (pos {})\n", pos); + + if (debug_stream) + *debug_stream << fmt::format("inserted (pos {})\n", pos); } else { arr.insert(arr.begin(), getRandomField(0)); - std::cerr << "inserted (0)\n"; + + if (debug_stream) + *debug_stream << "inserted (0)\n"; } } @@ -344,7 +352,8 @@ void QueryFuzzer::fuzzOrderByList(IAST * ast) } else { - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } } @@ -378,7 +387,8 @@ void QueryFuzzer::fuzzColumnLikeExpressionList(IAST * ast) if (col) impl->children.insert(pos, col); else - std::cerr << "No random column.\n"; + if (debug_stream) + *debug_stream << "No random column.\n"; } // We don't have to recurse here to fuzz the children, this is handled by @@ -1361,11 +1371,15 @@ void QueryFuzzer::fuzzMain(ASTPtr & ast) collectFuzzInfoMain(ast); fuzz(ast); - std::cout << std::endl; - WriteBufferFromOStream ast_buf(std::cout, 4096); - formatAST(*ast, ast_buf, false /*highlight*/); - ast_buf.finalize(); - std::cout << std::endl << std::endl; + if (out_stream) + { + *out_stream << std::endl; + + WriteBufferFromOStream ast_buf(*out_stream, 4096); + formatAST(*ast, ast_buf, false /*highlight*/); + ast_buf.finalize(); + *out_stream << std::endl << std::endl; + } } } diff --git a/src/Client/QueryFuzzer.h b/src/Common/QueryFuzzer.h similarity index 91% rename from src/Client/QueryFuzzer.h rename to src/Common/QueryFuzzer.h index 6165e589cae..35d088809f2 100644 --- a/src/Client/QueryFuzzer.h +++ b/src/Common/QueryFuzzer.h @@ -35,9 +35,31 @@ struct ASTWindowDefinition; * queries, so you want to feed it a lot of queries to get some interesting mix * of them. Normally we feed SQL regression tests to it. */ -struct QueryFuzzer +class QueryFuzzer { - pcg64 fuzz_rand{randomSeed()}; +public: + explicit QueryFuzzer(pcg64 fuzz_rand_ = randomSeed(), std::ostream * out_stream_ = nullptr, std::ostream * debug_stream_ = nullptr) + : fuzz_rand(fuzz_rand_) + , out_stream(out_stream_) + , debug_stream(debug_stream_) + { + } + + // This is the only function you have to call -- it will modify the passed + // ASTPtr to point to new AST with some random changes. + void fuzzMain(ASTPtr & ast); + + ASTs getInsertQueriesForFuzzedTables(const String & full_query); + ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); + void notifyQueryFailed(ASTPtr ast); + + static bool isSuitableForFuzzing(const ASTCreateQuery & create); + +private: + pcg64 fuzz_rand; + + std::ostream * out_stream = nullptr; + std::ostream * debug_stream = nullptr; // We add elements to expression lists with fixed probability. Some elements // are so large, that the expected number of elements we add to them is @@ -66,10 +88,6 @@ struct QueryFuzzer std::unordered_map index_of_fuzzed_table; std::set created_tables_hashes; - // This is the only function you have to call -- it will modify the passed - // ASTPtr to point to new AST with some random changes. - void fuzzMain(ASTPtr & ast); - // Various helper functions follow, normally you shouldn't have to call them. Field getRandomField(int type); Field fuzzField(Field field); @@ -77,9 +95,6 @@ struct QueryFuzzer ASTPtr getRandomExpressionList(); DataTypePtr fuzzDataType(DataTypePtr type); DataTypePtr getRandomType(); - ASTs getInsertQueriesForFuzzedTables(const String & full_query); - ASTs getDropQueriesForFuzzedTables(const ASTDropQuery & drop_query); - void notifyQueryFailed(ASTPtr ast); void replaceWithColumnLike(ASTPtr & ast); void replaceWithTableLike(ASTPtr & ast); void fuzzOrderByElement(ASTOrderByElement * elem); @@ -102,8 +117,6 @@ struct QueryFuzzer void addTableLike(ASTPtr ast); void addColumnLike(ASTPtr ast); void collectFuzzInfoRecurse(ASTPtr ast); - - static bool isSuitableForFuzzing(const ASTCreateQuery & create); }; } diff --git a/src/Storages/StorageFuzzQuery.cpp b/src/Storages/StorageFuzzQuery.cpp new file mode 100644 index 00000000000..6e8f425f8dc --- /dev/null +++ b/src/Storages/StorageFuzzQuery.cpp @@ -0,0 +1,169 @@ +#include + +#include +#include +#include +#include +#include +#include +#include +#include +#include + +namespace DB +{ + +namespace ErrorCodes +{ + extern const int BAD_ARGUMENTS; + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +ColumnPtr FuzzQuerySource::createColumn() +{ + auto column = ColumnString::create(); + ColumnString::Chars & data_to = column->getChars(); + ColumnString::Offsets & offsets_to = column->getOffsets(); + + offsets_to.resize(block_size); + IColumn::Offset offset = 0; + + auto fuzz_base = query; + size_t row_num = 0; + + while (row_num < block_size) + { + ASTPtr new_query = fuzz_base->clone(); + + auto base_before_fuzz = fuzz_base->formatForErrorMessage(); + fuzzer.fuzzMain(new_query); + auto fuzzed_text = new_query->formatForErrorMessage(); + + if (base_before_fuzz == fuzzed_text) + continue; + + /// AST is too long, will start from the original query. + if (config.max_query_length > 500) + { + fuzz_base = query; + continue; + } + + IColumn::Offset next_offset = offset + fuzzed_text.size() + 1; + data_to.resize(next_offset); + + std::copy(fuzzed_text.begin(), fuzzed_text.end(), &data_to[offset]); + + data_to[offset + fuzzed_text.size()] = 0; + offsets_to[row_num] = next_offset; + + offset = next_offset; + fuzz_base = new_query; + ++row_num; + } + + return column; +} + +StorageFuzzQuery::StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_) + : IStorage(table_id_), config(config_) +{ + StorageInMemoryMetadata storage_metadata; + storage_metadata.setColumns(columns_); + storage_metadata.setComment(comment_); + setInMemoryMetadata(storage_metadata); +} + +Pipe StorageFuzzQuery::read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & /*query_info*/, + ContextPtr /*context*/, + QueryProcessingStage::Enum /*processed_stage*/, + size_t max_block_size, + size_t num_streams) +{ + storage_snapshot->check(column_names); + + Pipes pipes; + pipes.reserve(num_streams); + + const ColumnsDescription & our_columns = storage_snapshot->metadata->getColumns(); + Block block_header; + for (const auto & name : column_names) + { + const auto & name_type = our_columns.get(name); + MutableColumnPtr column = name_type.type->createColumn(); + block_header.insert({std::move(column), name_type.type, name_type.name}); + } + + const char * begin = config.query.data(); + const char * end = begin + config.query.size(); + + ParserQuery parser(end, false); + auto query = parseQuery(parser, begin, end, "", 0, DBMS_DEFAULT_MAX_PARSER_DEPTH, DBMS_DEFAULT_MAX_PARSER_BACKTRACKS); + + for (UInt64 i = 0; i < num_streams; ++i) + pipes.emplace_back(std::make_shared(max_block_size, block_header, config, query)); + + return Pipe::unitePipes(std::move(pipes)); +} + +StorageFuzzQuery::Configuration StorageFuzzQuery::getConfiguration(ASTs & engine_args, ContextPtr local_context) +{ + StorageFuzzQuery::Configuration configuration{}; + + // Supported signatures: + // + // FuzzQuery(query) + // FuzzQuery(query, max_query_length) + // FuzzQuery(query, max_query_length, random_seed) + if (engine_args.empty() || engine_args.size() > 3) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "FuzzQuery requires 1 to 3 arguments: query, max_query_length, random_seed"); + + for (auto & engine_arg : engine_args) + engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, local_context); + + auto first_arg = checkAndGetLiteralArgument(engine_args[0], "query"); + configuration.query = std::move(first_arg); + + if (engine_args.size() >= 2) + { + const auto & literal = engine_args[1]->as(); + if (!literal.value.isNull()) + configuration.max_query_length = checkAndGetLiteralArgument(literal, "max_query_length"); + } + + if (engine_args.size() == 3) + { + const auto & literal = engine_args[2]->as(); + if (!literal.value.isNull()) + configuration.random_seed = checkAndGetLiteralArgument(literal, "random_seed"); + } + + return configuration; +} + +void registerStorageFuzzQuery(StorageFactory & factory) +{ + factory.registerStorage( + "FuzzQuery", + [](const StorageFactory::Arguments & args) -> std::shared_ptr + { + ASTs & engine_args = args.engine_args; + + if (engine_args.empty()) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Storage FuzzQuery must have arguments."); + + StorageFuzzQuery::Configuration configuration = StorageFuzzQuery::getConfiguration(engine_args, args.getLocalContext()); + + for (const auto& col : args.columns) + if (col.type->getTypeId() != TypeIndex::String) + throw Exception(ErrorCodes::BAD_ARGUMENTS, "'StorageFuzzQuery' supports only columns of String type, got {}.", col.type->getName()); + + return std::make_shared(args.table_id, args.columns, args.comment, configuration); + }); +} + +} diff --git a/src/Storages/StorageFuzzQuery.h b/src/Storages/StorageFuzzQuery.h new file mode 100644 index 00000000000..125ef960e74 --- /dev/null +++ b/src/Storages/StorageFuzzQuery.h @@ -0,0 +1,88 @@ +#pragma once + +#include +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class NamedCollection; + +class StorageFuzzQuery final : public IStorage +{ +public: + struct Configuration : public StatelessTableEngineConfiguration + { + String query; + UInt64 max_query_length = 500; + UInt64 random_seed = randomSeed(); + }; + + StorageFuzzQuery( + const StorageID & table_id_, const ColumnsDescription & columns_, const String & comment_, const Configuration & config_); + + std::string getName() const override { return "FuzzQuery"; } + + Pipe read( + const Names & column_names, + const StorageSnapshotPtr & storage_snapshot, + SelectQueryInfo & query_info, + ContextPtr context, + QueryProcessingStage::Enum processed_stage, + size_t max_block_size, + size_t num_streams) override; + + static StorageFuzzQuery::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context); + +private: + const Configuration config; +}; + + +class FuzzQuerySource : public ISource +{ +public: + FuzzQuerySource( + UInt64 block_size_, Block block_header_, const StorageFuzzQuery::Configuration & config_, ASTPtr query_) + : ISource(block_header_) + , block_size(block_size_) + , block_header(std::move(block_header_)) + , config(config_) + , query(query_) + , fuzzer(config_.random_seed) + { + } + + String getName() const override { return "FuzzQuery"; } + +protected: + Chunk generate() override + { + Columns columns; + columns.reserve(block_header.columns()); + for (const auto & col : block_header) + { + chassert(col.type->getTypeId() == TypeIndex::String); + columns.emplace_back(createColumn()); + } + + return {std::move(columns), block_size}; + } + +private: + ColumnPtr createColumn(); + + UInt64 block_size; + Block block_header; + + StorageFuzzQuery::Configuration config; + ASTPtr query; + + QueryFuzzer fuzzer; +}; + +} diff --git a/src/Storages/registerStorages.cpp b/src/Storages/registerStorages.cpp index 8f33314397c..adc1074b1fe 100644 --- a/src/Storages/registerStorages.cpp +++ b/src/Storages/registerStorages.cpp @@ -26,6 +26,7 @@ void registerStorageGenerateRandom(StorageFactory & factory); void registerStorageExecutable(StorageFactory & factory); void registerStorageWindowView(StorageFactory & factory); void registerStorageLoop(StorageFactory & factory); +void registerStorageFuzzQuery(StorageFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerStorageFuzzJSON(StorageFactory & factory); #endif @@ -126,6 +127,7 @@ void registerStorages() registerStorageExecutable(factory); registerStorageWindowView(factory); registerStorageLoop(factory); + registerStorageFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerStorageFuzzJSON(factory); #endif diff --git a/src/TableFunctions/TableFunctionFuzzQuery.cpp b/src/TableFunctions/TableFunctionFuzzQuery.cpp new file mode 100644 index 00000000000..224f6666556 --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.cpp @@ -0,0 +1,54 @@ +#include + +#include +#include +#include +#include + +namespace DB +{ + + +namespace ErrorCodes +{ + extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; +} + +void TableFunctionFuzzQuery::parseArguments(const ASTPtr & ast_function, ContextPtr context) +{ + ASTs & args_func = ast_function->children; + + if (args_func.size() != 1) + throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' must have arguments", getName()); + + auto args = args_func.at(0)->children; + configuration = StorageFuzzQuery::getConfiguration(args, context); +} + +StoragePtr TableFunctionFuzzQuery::executeImpl( + const ASTPtr & /*ast_function*/, + ContextPtr context, + const std::string & table_name, + ColumnsDescription /*cached_columns*/, + bool is_insert_query) const +{ + ColumnsDescription columns = getActualTableStructure(context, is_insert_query); + auto res = std::make_shared( + StorageID(getDatabaseName(), table_name), + columns, + /* comment */ String{}, + configuration); + res->startup(); + return res; +} + +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory) +{ + factory.registerFunction( + {.documentation + = {.description = "Perturbs a query string with random variations.", + .returned_value = "A table object with a single column containing perturbed query strings."}, + .allow_readonly = true}); +} + +} diff --git a/src/TableFunctions/TableFunctionFuzzQuery.h b/src/TableFunctions/TableFunctionFuzzQuery.h new file mode 100644 index 00000000000..22d10341c4d --- /dev/null +++ b/src/TableFunctions/TableFunctionFuzzQuery.h @@ -0,0 +1,42 @@ +#pragma once + +#include + +#include +#include +#include + +#include "config.h" + +namespace DB +{ + +class TableFunctionFuzzQuery : public ITableFunction +{ +public: + static constexpr auto name = "fuzzQuery"; + std::string getName() const override { return name; } + + void parseArguments(const ASTPtr & ast_function, ContextPtr context) override; + + ColumnsDescription getActualTableStructure(ContextPtr /* context */, bool /* is_insert_query */) const override + { + return ColumnsDescription{{"query", std::make_shared()}}; + } + +private: + StoragePtr executeImpl( + const ASTPtr & ast_function, + ContextPtr context, + const std::string & table_name, + ColumnsDescription cached_columns, + bool is_insert_query) const override; + + const char * getStorageTypeName() const override { return "fuzzQuery"; } + + String source; + std::optional random_seed; + StorageFuzzQuery::Configuration configuration; +}; + +} diff --git a/src/TableFunctions/registerTableFunctions.cpp b/src/TableFunctions/registerTableFunctions.cpp index ca4913898f9..a6c90872f12 100644 --- a/src/TableFunctions/registerTableFunctions.cpp +++ b/src/TableFunctions/registerTableFunctions.cpp @@ -26,6 +26,7 @@ void registerTableFunctions() registerTableFunctionMongoDB(factory); registerTableFunctionRedis(factory); registerTableFunctionMergeTreeIndex(factory); + registerTableFunctionFuzzQuery(factory); #if USE_RAPIDJSON || USE_SIMDJSON registerTableFunctionFuzzJSON(factory); #endif diff --git a/src/TableFunctions/registerTableFunctions.h b/src/TableFunctions/registerTableFunctions.h index efde4d6dcdc..2a8864a9bfd 100644 --- a/src/TableFunctions/registerTableFunctions.h +++ b/src/TableFunctions/registerTableFunctions.h @@ -23,6 +23,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory); void registerTableFunctionMongoDB(TableFunctionFactory & factory); void registerTableFunctionRedis(TableFunctionFactory & factory); void registerTableFunctionMergeTreeIndex(TableFunctionFactory & factory); +void registerTableFunctionFuzzQuery(TableFunctionFactory & factory); #if USE_RAPIDJSON || USE_SIMDJSON void registerTableFunctionFuzzJSON(TableFunctionFactory & factory); #endif diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.reference b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference new file mode 100644 index 00000000000..202e4557a33 --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.reference @@ -0,0 +1,2 @@ +query +String diff --git a/tests/queries/0_stateless/03031_table_function_fuzzquery.sql b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql new file mode 100644 index 00000000000..b26096f7f0e --- /dev/null +++ b/tests/queries/0_stateless/03031_table_function_fuzzquery.sql @@ -0,0 +1,18 @@ + +SELECT * FROM fuzzQuery('SELECT 1', 500, 8956) LIMIT 0 FORMAT TSVWithNamesAndTypes; + +SELECT * FROM fuzzQuery('SELECT * +FROM ( + SELECT + ([toString(number % 2)] :: Array(LowCardinality(String))) AS item_id, + count() + FROM numbers(3) + GROUP BY item_id WITH TOTALS +) AS l FULL JOIN ( + SELECT + ([toString((number % 2) * 2)] :: Array(String)) AS item_id + FROM numbers(3) +) AS r +ON l.item_id = r.item_id +ORDER BY 1,2,3; +', 500, 8956) LIMIT 10 FORMAT NULL; From 08bde9cb44dce6fdf069527852faa8ec29a71b10 Mon Sep 17 00:00:00 2001 From: pufit Date: Fri, 2 Aug 2024 18:28:33 -0400 Subject: [PATCH 2/2] fix conflict --- src/Client/ClientBase.h | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/src/Client/ClientBase.h b/src/Client/ClientBase.h index 1a23b6b1363..45251aea28a 100644 --- a/src/Client/ClientBase.h +++ b/src/Client/ClientBase.h @@ -2,7 +2,7 @@ #include -#include +#include #include #include #include