add fuzzer for table definition

This commit is contained in:
Anton Popov 2022-08-11 02:34:10 +00:00
parent 465cc7807a
commit e347fa2f45
7 changed files with 390 additions and 66 deletions

View File

@ -0,0 +1,6 @@
<clickhouse>
<!-- Allow nullable key to avoid errors while fuzzing definitions of tables -->
<merge_tree>
<allow_nullable_key>1</allow_nullable_key>
</merge_tree>
</clickhouse>

View File

@ -85,6 +85,7 @@ function configure
# TODO figure out which ones are needed
cp -av --dereference "$repo_dir"/tests/config/config.d/listen.xml db/config.d
cp -av --dereference "$script_dir"/query-fuzzer-tweaks-users.xml db/users.d
cp -av --dereference "$script_dir"/allow-nullable-key.xml db/config.d
}
function watchdog

View File

@ -12,6 +12,7 @@
#include <string>
#include "Client.h"
#include "Core/Protocol.h"
#include "Parsers/formatAST.h"
#include <base/find_symbols.h>
@ -513,6 +514,66 @@ static bool queryHasWithClause(const IAST & ast)
return false;
}
std::optional<bool> Client::processFuzzingStep(const String & query_to_execute, const ASTPtr & parsed_query)
{
processParsedSingleQuery(query_to_execute, query_to_execute, parsed_query);
const auto * exception = server_exception ? server_exception.get() : client_exception.get();
// Sometimes you may get TOO_DEEP_RECURSION from the server,
// and TOO_DEEP_RECURSION should not fail the fuzzer check.
if (have_error && exception->code() == ErrorCodes::TOO_DEEP_RECURSION)
{
have_error = false;
server_exception.reset();
client_exception.reset();
return true;
}
if (have_error)
{
fmt::print(stderr, "Error on processing query '{}': {}\n", parsed_query->formatForErrorMessage(), exception->message());
// Try to reconnect after errors, for two reasons:
// 1. We might not have realized that the server died, e.g. if
// it sent us a <Fatal> trace and closed connection properly.
// 2. The connection might have gotten into a wrong state and
// the next query will get false positive about
// "Unknown packet from server".
try
{
connection->forceConnected(connection_parameters.timeouts);
}
catch (...)
{
// Just report it, we'll terminate below.
fmt::print(stderr,
"Error while reconnecting to the server: {}\n",
getCurrentExceptionMessage(true));
// The reconnection might fail, but we'll still be connected
// in the sense of `connection->isConnected() = true`,
// in case when the requested database doesn't exist.
// Disconnect manually now, so that the following code doesn't
// have any doubts, and the connection state is predictable.
connection->disconnect();
}
}
if (!connection->isConnected())
{
// Probably the server is dead because we found an assertion
// failure. Fail fast.
fmt::print(stderr, "Lost connection to the server.\n");
// Print the changed settings because they might be needed to
// reproduce the error.
printChangedSettings();
return false;
}
return std::nullopt;
}
/// Returns false when server is not available.
bool Client::processWithFuzzing(const String & full_query)
@ -557,18 +618,28 @@ bool Client::processWithFuzzing(const String & full_query)
// - SET -- The time to fuzz the settings has not yet come
// (see comments in Client/QueryFuzzer.cpp)
size_t this_query_runs = query_fuzzer_runs;
if (orig_ast->as<ASTInsertQuery>() ||
orig_ast->as<ASTCreateQuery>() ||
orig_ast->as<ASTDropQuery>() ||
orig_ast->as<ASTSetQuery>())
ASTs inserts_for_fuzzed_tables;
if (orig_ast->as<ASTDropQuery>() || orig_ast->as<ASTSetQuery>())
{
this_query_runs = 1;
}
else if (const auto * create = orig_ast->as<ASTCreateQuery>())
{
if (create->columns_list)
this_query_runs = create_query_fuzzer_runs;
else
this_query_runs = 1;
}
else if (const auto * insert = orig_ast->as<ASTInsertQuery>())
{
this_query_runs = 1;
inserts_for_fuzzed_tables = fuzzer.getInsertQueriesForFuzzedTables(full_query);
}
String query_to_execute;
ASTPtr parsed_query;
ASTPtr fuzz_base = orig_ast;
for (size_t fuzz_step = 0; fuzz_step < this_query_runs; ++fuzz_step)
{
fmt::print(stderr, "Fuzzing step {} out of {}\n", fuzz_step, this_query_runs);
@ -629,9 +700,9 @@ bool Client::processWithFuzzing(const String & full_query)
continue;
}
parsed_query = ast_to_process;
query_to_execute = parsed_query->formatForErrorMessage();
processParsedSingleQuery(full_query, query_to_execute, parsed_query);
query_to_execute = ast_to_process->formatForErrorMessage();
if (auto res = processFuzzingStep(query_to_execute, ast_to_process))
return *res;
}
catch (...)
{
@ -644,60 +715,6 @@ bool Client::processWithFuzzing(const String & full_query)
have_error = true;
}
const auto * exception = server_exception ? server_exception.get() : client_exception.get();
// Sometimes you may get TOO_DEEP_RECURSION from the server,
// and TOO_DEEP_RECURSION should not fail the fuzzer check.
if (have_error && exception->code() == ErrorCodes::TOO_DEEP_RECURSION)
{
have_error = false;
server_exception.reset();
client_exception.reset();
return true;
}
if (have_error)
{
fmt::print(stderr, "Error on processing query '{}': {}\n", ast_to_process->formatForErrorMessage(), exception->message());
// Try to reconnect after errors, for two reasons:
// 1. We might not have realized that the server died, e.g. if
// it sent us a <Fatal> trace and closed connection properly.
// 2. The connection might have gotten into a wrong state and
// the next query will get false positive about
// "Unknown packet from server".
try
{
connection->forceConnected(connection_parameters.timeouts);
}
catch (...)
{
// Just report it, we'll terminate below.
fmt::print(stderr,
"Error while reconnecting to the server: {}\n",
getCurrentExceptionMessage(true));
// The reconnection might fail, but we'll still be connected
// in the sense of `connection->isConnected() = true`,
// in case when the requested database doesn't exist.
// Disconnect manually now, so that the following code doesn't
// have any doubts, and the connection state is predictable.
connection->disconnect();
}
}
if (!connection->isConnected())
{
// Probably the server is dead because we found an assertion
// failure. Fail fast.
fmt::print(stderr, "Lost connection to the server.\n");
// Print the changed settings because they might be needed to
// reproduce the error.
printChangedSettings();
return false;
}
// Check that after the query is formatted, we can parse it back,
// format again and get the same result. Unfortunately, we can't
// compare the ASTs, which would be more sensitive to errors. This
@ -728,13 +745,12 @@ bool Client::processWithFuzzing(const String & full_query)
// query, but second and third.
// If you have to add any more workarounds to this check, just remove
// it altogether, it's not so useful.
if (parsed_query && !have_error && !queryHasWithClause(*parsed_query))
if (ast_to_process && !have_error && !queryHasWithClause(*ast_to_process))
{
ASTPtr ast_2;
try
{
const auto * tmp_pos = query_to_execute.c_str();
ast_2 = parseQuery(tmp_pos, tmp_pos + query_to_execute.size(), false /* allow_multi_statements */);
}
catch (Exception & e)
@ -761,7 +777,7 @@ bool Client::processWithFuzzing(const String & full_query)
"Got the following (different) text after formatting the fuzzed query and parsing it back:\n'{}'\n, expected:\n'{}'\n",
text_3, text_2);
fmt::print(stderr, "In more detail:\n");
fmt::print(stderr, "AST-1 (generated by fuzzer):\n'{}'\n", parsed_query->dumpTree());
fmt::print(stderr, "AST-1 (generated by fuzzer):\n'{}'\n", ast_to_process->dumpTree());
fmt::print(stderr, "Text-1 (AST-1 formatted):\n'{}'\n", query_to_execute);
fmt::print(stderr, "AST-2 (Text-1 parsed):\n'{}'\n", ast_2->dumpTree());
fmt::print(stderr, "Text-2 (AST-2 formatted):\n'{}'\n", text_2);
@ -799,6 +815,34 @@ bool Client::processWithFuzzing(const String & full_query)
}
}
for (const auto & insert_query : inserts_for_fuzzed_tables)
{
std::cout << std::endl;
WriteBufferFromOStream ast_buf(std::cout, 4096);
formatAST(*insert_query, ast_buf, false /*highlight*/);
ast_buf.next();
std::cout << std::endl << std::endl;
try
{
query_to_execute = insert_query->formatForErrorMessage();
if (auto res = processFuzzingStep(query_to_execute, insert_query))
return *res;
}
catch (...)
{
client_exception = std::make_unique<Exception>(getCurrentExceptionMessage(print_stack_trace), getCurrentExceptionCode());
have_error = true;
}
if (have_error)
{
server_exception.reset();
client_exception.reset();
have_error = false;
}
}
return true;
}
@ -833,6 +877,7 @@ void Client::addOptions(OptionsDescription & options_description)
("compression", po::value<bool>(), "enable or disable compression (enabled by default for remote communication and disabled for localhost communication).")
("query-fuzzer-runs", po::value<int>()->default_value(0), "After executing every SELECT query, do random mutations in it and run again specified number of times. This is used for testing to discover unexpected corner cases.")
("create-query-fuzzer-runs", po::value<int>()->default_value(0), "")
("interleave-queries-file", po::value<std::vector<std::string>>()->multitoken(),
"file path with queries to execute before every file from 'queries-file'; multiple files can be specified (--queries-file file1 file2...); this is needed to enable more aggressive fuzzing of newly added tests (see 'query-fuzzer-runs' option)")
@ -985,6 +1030,17 @@ void Client::processOptions(const OptionsDescription & options_description,
ignore_error = true;
}
if ((create_query_fuzzer_runs = options["create-query-fuzzer-runs"].as<int>()))
{
// Fuzzer implies multiquery.
config().setBool("multiquery", true);
// Ignore errors in parsing queries.
config().setBool("ignore-error", true);
global_context->setSetting("allow_suspicious_low_cardinality_types", true);
ignore_error = true;
}
if (options.count("opentelemetry-traceparent"))
{
String traceparent = options["opentelemetry-traceparent"].as<std::string>();

View File

@ -17,6 +17,7 @@ public:
protected:
bool processWithFuzzing(const String & full_query) override;
std::optional<bool> processFuzzingStep(const String & query_to_execute, const ASTPtr & parsed_query);
void connect() override;

View File

@ -247,6 +247,7 @@ protected:
QueryFuzzer fuzzer;
int query_fuzzer_runs = 0;
int create_query_fuzzer_runs = 0;
struct
{

View File

@ -1,4 +1,21 @@
#include "QueryFuzzer.h"
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeFixedString.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypesDecimal.h>
#include <DataTypes/IDataType.h>
#include <IO/ReadBufferFromString.h>
#include <IO/WriteHelpers.h>
#include <Parsers/ASTColumnDeclaration.h>
#include <Parsers/ASTCreateQuery.h>
#include <Parsers/IAST_fwd.h>
#include <Parsers/ParserDataType.h>
#include <Parsers/ParserInsertQuery.h>
#include <unordered_set>
@ -35,6 +52,7 @@ namespace DB
namespace ErrorCodes
{
extern const int TOO_DEEP_RECURSION;
extern const int LOGICAL_ERROR;
}
Field QueryFuzzer::getRandomField(int type)
@ -398,6 +416,228 @@ void QueryFuzzer::fuzzWindowFrame(ASTWindowDefinition & def)
}
}
void QueryFuzzer::fuzzCreateQuery(ASTCreateQuery & create)
{
if (create.columns_list && create.columns_list->columns)
{
for (auto & ast : create.columns_list->columns->children)
{
if (auto * column = ast->as<ASTColumnDeclaration>())
{
fuzzColumnDeclaration(*column);
}
}
}
if (create.storage && create.storage->engine)
{
auto & engine_name = create.storage->engine->name;
if (startsWith(engine_name, "Replicated"))
engine_name = engine_name.substr(strlen("Replicated"));
}
auto full_name = create.getTable();
auto original_name = full_name.substr(0, full_name.find("__fuzz_"));
size_t index = index_of_fuzzed_table[original_name]++;
auto new_name = original_name + "__fuzz_" + toString(index);
create.setTable(new_name);
SipHash sip_hash;
sip_hash.update(original_name);
if (create.columns_list)
create.columns_list->updateTreeHash(sip_hash);
if (create.storage)
create.columns_list->updateTreeHash(sip_hash);
IAST::Hash hash;
sip_hash.get128(hash);
if (created_tables_hashes.insert(hash).second)
original_table_name_to_fuzzed[original_name].push_back(new_name);
}
void QueryFuzzer::fuzzColumnDeclaration(ASTColumnDeclaration & column)
{
if (column.type)
{
auto data_type = fuzzDataType(DataTypeFactory::instance().get(column.type));
ParserDataType parser;
column.type = parseQuery(parser, data_type->getName(), DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH);
}
}
DataTypePtr QueryFuzzer::fuzzDataType(DataTypePtr type)
{
/// Do not replace Array with not Array to often.
const auto * type_array = typeid_cast<const DataTypeArray *>(type.get());
if (type_array && fuzz_rand() % 5 != 0)
return std::make_shared<DataTypeArray>(fuzzDataType(type_array->getNestedType()));
const auto * type_tuple = typeid_cast<const DataTypeTuple *>(type.get());
if (type_tuple && fuzz_rand() % 5 != 0)
{
DataTypes elements;
for (const auto & element : type_tuple->getElements())
elements.push_back(fuzzDataType(element));
return type_tuple->haveExplicitNames()
? std::make_shared<DataTypeTuple>(elements, type_tuple->getElementNames())
: std::make_shared<DataTypeTuple>(elements);
}
const auto * type_map = typeid_cast<const DataTypeMap *>(type.get());
if (type_map && fuzz_rand() % 5 != 0)
{
auto key_type = fuzzDataType(type_map->getKeyType());
auto value_type = fuzzDataType(type_map->getValueType());
if (!DataTypeMap::checkKeyType(key_type))
key_type = type_map->getKeyType();
return std::make_shared<DataTypeMap>(key_type, value_type);
}
const auto * type_nullable = typeid_cast<const DataTypeNullable *>(type.get());
if (type_nullable)
{
size_t tmp = fuzz_rand() % 3;
if (tmp == 0)
return type_nullable->getNestedType();
if (tmp == 1)
{
auto nested_type = fuzzDataType(type_nullable->getNestedType());
if (nested_type->canBeInsideNullable())
return std::make_shared<DataTypeNullable>(nested_type);
}
}
const auto * type_low_cardinality = typeid_cast<const DataTypeLowCardinality *>(type.get());
if (type_low_cardinality)
{
size_t tmp = fuzz_rand() % 3;
if (tmp == 0)
return type_low_cardinality->getDictionaryType();
if (tmp == 1)
{
auto nested_type = fuzzDataType(type_low_cardinality->getDictionaryType());
if (nested_type->canBeInsideLowCardinality())
return std::make_shared<DataTypeLowCardinality>(nested_type);
}
}
size_t tmp = fuzz_rand() % 10;
if (tmp <= 1 && type->canBeInsideNullable())
return std::make_shared<DataTypeNullable>(type);
if (tmp <= 3 && type->canBeInsideLowCardinality())
return std::make_shared<DataTypeLowCardinality>(type);
if (tmp == 4)
return getRandomType();
return type;
}
DataTypePtr QueryFuzzer::getRandomType()
{
auto type_id = static_cast<TypeIndex>(fuzz_rand() % static_cast<size_t>(TypeIndex::Tuple) + 1);
if (type_id == TypeIndex::Tuple)
{
size_t tuple_size = fuzz_rand() % 6 + 1;
DataTypes elements;
for (size_t i = 0; i < tuple_size; ++i)
elements.push_back(getRandomType());
return std::make_shared<DataTypeTuple>(elements);
}
if (type_id == TypeIndex::Array)
return std::make_shared<DataTypeArray>(getRandomType());
#define DISPATCH(DECIMAL) \
if (type_id == TypeIndex::DECIMAL) \
return std::make_shared<DataTypeDecimal<DECIMAL>>( \
DataTypeDecimal<DECIMAL>::maxPrecision(), DataTypeDecimal<DECIMAL>::maxPrecision()); // NOLINT
DISPATCH(Decimal32)
DISPATCH(Decimal64)
DISPATCH(Decimal128)
DISPATCH(Decimal256)
#undef DISPATCH
if (type_id == TypeIndex::FixedString)
return std::make_shared<DataTypeFixedString>(fuzz_rand() % 20);
if (type_id == TypeIndex::Enum8)
return std::make_shared<DataTypeUInt8>();
if (type_id == TypeIndex::Enum16)
return std::make_shared<DataTypeUInt16>();
return DataTypeFactory::instance().get(String(magic_enum::enum_name(type_id)));
}
void QueryFuzzer::fuzzTableName(ASTTableExpression & table)
{
if (!table.database_and_table_name || fuzz_rand() % 3 == 0)
return;
const auto * identifier = table.database_and_table_name->as<ASTTableIdentifier>();
if (!identifier)
return;
auto table_id = identifier->getTableId();
if (table_id.empty())
return;
auto it = original_table_name_to_fuzzed.find(table_id.getTableName());
if (it != original_table_name_to_fuzzed.end() && !it->second.empty())
{
const auto & new_table_name = it->second[fuzz_rand() % it->second.size()];
StorageID new_table_id(table_id.database_name, new_table_name);
table.database_and_table_name = std::make_shared<ASTTableIdentifier>(new_table_id);
}
}
static ASTPtr tryParseInsertQuery(const String & full_query)
{
const char * pos = full_query.data();
const char * end = full_query.data() + full_query.size();
ParserInsertQuery parser(end, false);
String message;
return tryParseQuery(parser, pos, end, message, false, "", false, DBMS_DEFAULT_MAX_QUERY_SIZE, DBMS_DEFAULT_MAX_PARSER_DEPTH);
}
ASTs QueryFuzzer::getInsertQueriesForFuzzedTables(const String & full_query)
{
auto parsed_query = tryParseInsertQuery(full_query);
if (!parsed_query)
return {};
const auto & insert = *parsed_query->as<ASTInsertQuery>();
if (!insert.table)
return {};
auto table_name = insert.getTable();
auto it = original_table_name_to_fuzzed.find(table_name);
if (it == original_table_name_to_fuzzed.end())
return {};
ASTs queries;
for (const auto & fuzzed_name : it->second)
{
auto & query = queries.emplace_back(tryParseInsertQuery(full_query));
query->as<ASTInsertQuery>()->setTable(fuzzed_name);
}
return queries;
}
void QueryFuzzer::fuzz(ASTs & asts)
{
for (auto & ast : asts)
@ -465,6 +705,7 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
}
else if (auto * table_expr = typeid_cast<ASTTableExpression *>(ast.get()))
{
fuzzTableName(*table_expr);
fuzz(table_expr->children);
}
else if (auto * expr_list = typeid_cast<ASTExpressionList *>(ast.get()))
@ -531,6 +772,10 @@ void QueryFuzzer::fuzz(ASTPtr & ast)
literal->value = fuzzField(literal->value);
}
}
else if (auto * create_query = typeid_cast<ASTCreateQuery *>(ast.get()))
{
fuzzCreateQuery(*create_query);
}
else
{
fuzz(ast->children);

View File

@ -1,5 +1,6 @@
#pragma once
#include <DataTypes/IDataType.h>
#include <unordered_set>
#include <unordered_map>
#include <vector>
@ -16,6 +17,10 @@ namespace DB
class ASTExpressionList;
class ASTOrderByElement;
class ASTCreateQuery;
class ASTInsertQuery;
class ASTColumnDeclaration;
struct ASTTableExpression;
struct ASTWindowDefinition;
/*
@ -54,6 +59,9 @@ struct QueryFuzzer
std::unordered_set<const IAST *> debug_visited_nodes;
ASTPtr * debug_top_ast = nullptr;
std::unordered_map<std::string, std::vector<std::string>> original_table_name_to_fuzzed;
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.
@ -63,12 +71,18 @@ struct QueryFuzzer
Field getRandomField(int type);
Field fuzzField(Field field);
ASTPtr getRandomColumnLike();
DataTypePtr fuzzDataType(DataTypePtr type);
DataTypePtr getRandomType();
ASTs getInsertQueriesForFuzzedTables(const String & full_query);
void replaceWithColumnLike(ASTPtr & ast);
void replaceWithTableLike(ASTPtr & ast);
void fuzzOrderByElement(ASTOrderByElement * elem);
void fuzzOrderByList(IAST * ast);
void fuzzColumnLikeExpressionList(IAST * ast);
void fuzzWindowFrame(ASTWindowDefinition & def);
void fuzzCreateQuery(ASTCreateQuery & create);
void fuzzColumnDeclaration(ASTColumnDeclaration & column);
void fuzzTableName(ASTTableExpression & table);
void fuzz(ASTs & asts);
void fuzz(ASTPtr & ast);
void collectFuzzInfoMain(ASTPtr ast);