Merge pull request #62103 from ClickHouse/pufit/fuzz-query

FuzzQuery table function
This commit is contained in:
pufit 2024-07-04 20:46:47 +00:00 committed by GitHub
commit 0a5ee12f0b
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
15 changed files with 475 additions and 31 deletions

View File

@ -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 │
└────────────────────────────────────────────────────────────────┘
```

View File

@ -9,7 +9,10 @@ namespace DB
class Client : public ClientBase
{
public:
Client() = default;
Client()
{
fuzzer = QueryFuzzer(randomSeed(), &std::cout, &std::cerr);
}
void initialize(Poco::Util::Application & self) override;

View File

@ -6,13 +6,13 @@
#include <Common/ProgressIndication.h>
#include <Common/InterruptListener.h>
#include <Common/ShellCommand.h>
#include <Common/QueryFuzzer.h>
#include <Common/Stopwatch.h>
#include <Common/DNSResolver.h>
#include <Core/ExternalTable.h>
#include <Poco/Util/Application.h>
#include <Interpreters/Context.h>
#include <Client/Suggest.h>
#include <Client/QueryFuzzer.h>
#include <boost/program_options.hpp>
#include <Storages/StorageFile.h>
#include <Storages/SelectQueryInfo.h>

View File

@ -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<Decimal64>(
bad_int64_values[fuzz_rand() % (sizeof(bad_int64_values) / sizeof(*bad_int64_values))],
static_cast<UInt32>(scales[fuzz_rand() % (sizeof(scales) / sizeof(*scales))])
bad_int64_values[fuzz_rand() % std::size(bad_int64_values)],
static_cast<UInt32>(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;
}
}
}

View File

@ -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<std::string, size_t> index_of_fuzzed_table;
std::set<IAST::Hash> 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);
};
}

View File

@ -0,0 +1,169 @@
#include <Storages/StorageFuzzQuery.h>
#include <optional>
#include <unordered_set>
#include <Columns/ColumnString.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Storages/NamedCollectionsHelpers.h>
#include <Storages/StorageFactory.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Parsers/ParserQuery.h>
#include <Parsers/parseQuery.h>
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<FuzzQuerySource>(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<String>(engine_args[0], "query");
configuration.query = std::move(first_arg);
if (engine_args.size() >= 2)
{
const auto & literal = engine_args[1]->as<const ASTLiteral &>();
if (!literal.value.isNull())
configuration.max_query_length = checkAndGetLiteralArgument<UInt64>(literal, "max_query_length");
}
if (engine_args.size() == 3)
{
const auto & literal = engine_args[2]->as<const ASTLiteral &>();
if (!literal.value.isNull())
configuration.random_seed = checkAndGetLiteralArgument<UInt64>(literal, "random_seed");
}
return configuration;
}
void registerStorageFuzzQuery(StorageFactory & factory)
{
factory.registerStorage(
"FuzzQuery",
[](const StorageFactory::Arguments & args) -> std::shared_ptr<StorageFuzzQuery>
{
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<StorageFuzzQuery>(args.table_id, args.columns, args.comment, configuration);
});
}
}

View File

@ -0,0 +1,88 @@
#pragma once
#include <Storages/IStorage.h>
#include <Storages/StorageConfiguration.h>
#include <Common/randomSeed.h>
#include <Common/QueryFuzzer.h>
#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;
};
}

View File

@ -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
@ -123,6 +124,7 @@ void registerStorages()
registerStorageExecutable(factory);
registerStorageWindowView(factory);
registerStorageLoop(factory);
registerStorageFuzzQuery(factory);
#if USE_RAPIDJSON || USE_SIMDJSON
registerStorageFuzzJSON(factory);
#endif

View File

@ -0,0 +1,54 @@
#include <TableFunctions/TableFunctionFuzzQuery.h>
#include <DataTypes/DataTypeString.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/registerTableFunctions.h>
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<StorageFuzzQuery>(
StorageID(getDatabaseName(), table_name),
columns,
/* comment */ String{},
configuration);
res->startup();
return res;
}
void registerTableFunctionFuzzQuery(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionFuzzQuery>(
{.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});
}
}

View File

@ -0,0 +1,42 @@
#pragma once
#include <optional>
#include <TableFunctions/ITableFunction.h>
#include <DataTypes/DataTypeString.h>
#include <Storages/StorageFuzzQuery.h>
#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<DataTypeString>()}};
}
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<UInt64> random_seed;
StorageFuzzQuery::Configuration configuration;
};
}

View File

@ -26,6 +26,7 @@ void registerTableFunctions()
registerTableFunctionMongoDB(factory);
registerTableFunctionRedis(factory);
registerTableFunctionMergeTreeIndex(factory);
registerTableFunctionFuzzQuery(factory);
#if USE_RAPIDJSON || USE_SIMDJSON
registerTableFunctionFuzzJSON(factory);
#endif

View File

@ -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

View File

@ -0,0 +1,2 @@
query
String

View File

@ -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;

View File

@ -1658,6 +1658,7 @@ fuzzBits
fuzzJSON
fuzzer
fuzzers
fuzzQuery
gRPC
gccMurmurHash
gcem