Merge pull request #46971 from Avogar/better-hints

Better exception messages when schema_inference_hints is ill-formatted
This commit is contained in:
alesapin 2023-02-28 12:31:27 +01:00 committed by GitHub
commit 812423804c
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
7 changed files with 86 additions and 32 deletions

View File

@ -92,12 +92,11 @@ ColumnsDescription parseColumnsListFromString(const std::string & structure, con
return columns;
}
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context)
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context, String & error)
{
ParserColumnDeclarationList parser(true, true);
const Settings & settings = context->getSettingsRef();
String error;
const char * start = structure.data();
const char * end = structure.data() + structure.size();
ASTPtr columns_list_raw = tryParseQuery(parser, start, end, error, false, "columns declaration list", false, settings.max_query_size, settings.max_parser_depth);
@ -106,7 +105,10 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip
auto * columns_list = dynamic_cast<ASTExpressionList *>(columns_list_raw.get());
if (!columns_list)
{
error = fmt::format("Invalid columns declaration list: \"{}\"", structure);
return false;
}
try
{
@ -118,6 +120,7 @@ bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescrip
}
catch (...)
{
error = getCurrentExceptionMessage(false);
return false;
}
}

View File

@ -33,6 +33,6 @@ void validateDataType(const DataTypePtr & type, const DataTypeValidationSettings
/// Parses a common argument for table functions such as table structure given in string
ColumnsDescription parseColumnsListFromString(const std::string & structure, const ContextPtr & context);
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context);
bool tryParseColumnsListFromString(const std::string & structure, ColumnsDescription & columns, const ContextPtr & context, String & error);
}

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/SchemaInferenceUtils.h>
#include <DataTypes/DataTypeString.h>
#include <Common/logger_useful.h>
#include <Interpreters/parseColumnsListForTableFunction.h>
#include <boost/algorithm/string.hpp>
@ -15,20 +16,38 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
}
void checkFinalInferredType(DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read)
void checkFinalInferredType(
DataTypePtr & type,
const String & name,
const FormatSettings & settings,
const DataTypePtr & default_type,
size_t rows_read,
const String & hints_parsing_error)
{
if (!checkIfTypeIsComplete(type))
{
if (!default_type)
throw Exception(
ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA,
"Cannot determine type for column '{}' by first {} rows "
"of data, most likely this column contains only Nulls or empty "
"Arrays/Maps. You can specify the type for this column using setting schema_inference_hints. "
"If your data contains complex JSON objects, try enabling one "
"of the settings allow_experimental_object_type/input_format_json_read_objects_as_strings",
name,
rows_read);
{
if (hints_parsing_error.empty())
throw Exception(
ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA,
"Cannot determine type for column '{}' by first {} rows "
"of data, most likely this column contains only Nulls or empty "
"Arrays/Maps. You can specify the type for this column using setting schema_inference_hints. "
"If your data contains complex JSON objects, try enabling one "
"of the settings allow_experimental_object_type/input_format_json_read_objects_as_strings",
name,
rows_read);
else
throw Exception(
ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA,
"Cannot determine type for column '{}' by first {} rows "
"of data, most likely this column contains only Nulls or empty Arrays/Maps. "
"Column types from setting schema_inference_hints couldn't be parsed because of error: {}",
name,
rows_read,
hints_parsing_error);
}
type = default_type;
}
@ -46,11 +65,15 @@ IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & fo
void IIRowSchemaReader::setContext(ContextPtr & context)
{
ColumnsDescription columns;
if (tryParseColumnsListFromString(hints_str, columns, context))
if (tryParseColumnsListFromString(hints_str, columns, context, hints_parsing_error))
{
for (const auto & [name, type] : columns.getAll())
hints[name] = type;
}
else
{
LOG_WARNING(&Poco::Logger::get("IIRowSchemaReader"), "Couldn't parse schema inference hints: {}. This setting will be ignored", hints_parsing_error);
}
}
void IIRowSchemaReader::transformTypesIfNeeded(DataTypePtr & type, DataTypePtr & new_type)
@ -137,7 +160,14 @@ NamesAndTypesList IRowSchemaReader::readSchema()
if (!new_data_types[field_index] || hints.contains(column_names[field_index]))
continue;
chooseResultColumnType(*this, data_types[field_index], new_data_types[field_index], getDefaultType(field_index), std::to_string(field_index + 1), rows_read);
chooseResultColumnType(
*this,
data_types[field_index],
new_data_types[field_index],
getDefaultType(field_index),
std::to_string(field_index + 1),
rows_read,
hints_parsing_error);
}
}
@ -149,7 +179,7 @@ NamesAndTypesList IRowSchemaReader::readSchema()
{
transformFinalTypeIfNeeded(data_types[field_index]);
/// Check that we could determine the type of this column.
checkFinalInferredType(data_types[field_index], column_names[field_index], format_settings, getDefaultType(field_index), rows_read);
checkFinalInferredType(data_types[field_index], column_names[field_index], format_settings, getDefaultType(field_index), rows_read, hints_parsing_error);
}
result.emplace_back(column_names[field_index], data_types[field_index]);
}
@ -246,7 +276,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
continue;
auto & type = it->second;
chooseResultColumnType(*this, type, new_type, default_type, name, rows_read);
chooseResultColumnType(*this, type, new_type, default_type, name, rows_read, hints_parsing_error);
}
}
@ -263,7 +293,7 @@ NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
{
transformFinalTypeIfNeeded(type);
/// Check that we could determine the type of this column.
checkFinalInferredType(type, name, format_settings, default_type, rows_read);
checkFinalInferredType(type, name, format_settings, default_type, rows_read, hints_parsing_error);
}
result.emplace_back(name, type);
}

View File

@ -65,6 +65,7 @@ protected:
String hints_str;
FormatSettings format_settings;
std::unordered_map<String, DataTypePtr> hints;
String hints_parsing_error;
};
/// Base class for schema inference for formats that read data row by row.
@ -145,7 +146,8 @@ void chooseResultColumnType(
DataTypePtr & new_type,
const DataTypePtr & default_type,
const String & column_name,
size_t row)
size_t row,
const String & hints_parsing_error = "")
{
if (!type)
{
@ -166,14 +168,25 @@ void chooseResultColumnType(
type = default_type;
else
{
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. "
"You can specify the type for this column using setting schema_inference_hints",
type->getName(),
column_name,
row,
new_type->getName());
if (hints_parsing_error.empty())
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. "
"You can specify the type for this column using setting schema_inference_hints",
type->getName(),
column_name,
row,
new_type->getName());
else
throw Exception(
ErrorCodes::TYPE_MISMATCH,
"Automatically defined type {} for column '{}' in row {} differs from type defined by previous rows: {}. "
"Column types from setting schema_inference_hints couldn't be parsed because of error: {}",
type->getName(),
column_name,
row,
new_type->getName(),
hints_parsing_error);
}
}
@ -196,7 +209,13 @@ void chooseResultColumnTypes(
chooseResultColumnType(schema_reader, types[i], new_types[i], default_type, column_names[i], row);
}
void checkFinalInferredType(DataTypePtr & type, const String & name, const FormatSettings & settings, const DataTypePtr & default_type, size_t rows_read);
void checkFinalInferredType(
DataTypePtr & type,
const String & name,
const FormatSettings & settings,
const DataTypePtr & default_type,
size_t rows_read,
const String & hints_parsing_error);
Strings splitColumnNames(const String & column_names_str);

View File

@ -182,7 +182,7 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase(
void JSONColumnsSchemaReaderBase::setContext(ContextPtr & ctx)
{
ColumnsDescription columns;
if (tryParseColumnsListFromString(hints_str, columns, ctx))
if (tryParseColumnsListFromString(hints_str, columns, ctx, hints_parsing_error))
{
for (const auto & [name, type] : columns.getAll())
hints[name] = type;
@ -238,7 +238,7 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
rows_in_block = 0;
auto column_type = readColumnAndGetDataType(
column_name, rows_in_block, format_settings.max_rows_to_read_for_schema_inference - total_rows_read);
chooseResultColumnType(*this, names_to_types[column_name], column_type, nullptr, column_name, total_rows_read + 1);
chooseResultColumnType(*this, names_to_types[column_name], column_type, nullptr, column_name, total_rows_read + 1, hints_parsing_error);
}
++iteration;
@ -260,7 +260,7 @@ NamesAndTypesList JSONColumnsSchemaReaderBase::readSchema()
{
transformJSONTupleToArrayIfPossible(type, format_settings, &inference_info);
/// Check that we could determine the type of this column.
checkFinalInferredType(type, name, format_settings, nullptr, format_settings.max_rows_to_read_for_schema_inference);
checkFinalInferredType(type, name, format_settings, nullptr, format_settings.max_rows_to_read_for_schema_inference, hints_parsing_error);
}
result.emplace_back(name, type);
}

View File

@ -91,6 +91,7 @@ private:
const FormatSettings format_settings;
String hints_str;
std::unordered_map<String, DataTypePtr> hints;
String hints_parsing_error;
std::unique_ptr<JSONColumnsReaderBase> reader;
Names column_names_from_settings;
JSONInferenceInfo inference_info;

View File

@ -92,7 +92,8 @@ void TableFunctionValues::parseArguments(const ASTPtr & ast_function, ContextPtr
const auto & literal = args[0]->as<const ASTLiteral>();
String value;
if (args.size() > 1 && literal && literal->value.tryGet(value) && tryParseColumnsListFromString(value, structure, context))
String error;
if (args.size() > 1 && literal && literal->value.tryGet(value) && tryParseColumnsListFromString(value, structure, context, error))
{
has_structure_in_arguments = true;
return;