Implement schema inference for most input formats

This commit is contained in:
avogar 2021-12-15 14:30:57 +03:00
parent e671252131
commit 8112a71233
108 changed files with 4029 additions and 926 deletions

View File

@ -313,11 +313,11 @@ void LocalServer::cleanup()
std::string LocalServer::getInitialCreateTableQuery()
{
if (!config().has("table-structure"))
if (!config().has("table-structure") && !config().has("table-file"))
return {};
auto table_name = backQuoteIfNeed(config().getString("table-name", "table"));
auto table_structure = config().getString("table-structure");
auto table_structure = config().getString("table-structure", "auto");
auto data_format = backQuoteIfNeed(config().getString("table-data-format", "TSV"));
String table_file;
@ -332,7 +332,12 @@ std::string LocalServer::getInitialCreateTableQuery()
table_file = quoteString(config().getString("table-file"));
}
return fmt::format("CREATE TABLE {} ({}) ENGINE = File({}, {});",
if (table_structure == "auto")
table_structure = "";
else
table_structure = "(" + table_structure + ")";
return fmt::format("CREATE TABLE {} {} ENGINE = File({}, {});",
table_name, table_structure, data_format, table_file);
}
@ -422,7 +427,7 @@ try
#else
is_interactive = stdin_is_a_tty
&& (config().hasOption("interactive")
|| (!config().has("query") && !config().has("table-structure") && queries_files.empty()));
|| (!config().has("query") && !config().has("table-structure") && queries_files.empty() && !config().has("table-file")));
#endif
if (!is_interactive)
{

View File

@ -603,6 +603,7 @@
M(632, UNEXPECTED_DATA_AFTER_PARSED_VALUE) \
M(633, QUERY_IS_NOT_SUPPORTED_IN_WINDOW_VIEW) \
M(634, MONGODB_ERROR) \
M(635, CANNOT_EXTRACT_TABLE_STRUCTURE) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -26,6 +26,7 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
}
}
@ -1133,4 +1134,54 @@ Coordination::RequestPtr makeCheckRequest(const std::string & path, int version)
return request;
}
std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log)
{
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (!zookeeper_path.empty() && zookeeper_path.front() != '/')
{
/// Do not allow this for new tables, print warning for tables created in old versions
if (check_starts_with_slash)
throw DB::Exception(DB::ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must starts with '/', got '{}'", zookeeper_path);
if (log)
LOG_WARNING(log, "ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases");
zookeeper_path = "/" + zookeeper_path;
}
return zookeeper_path;
}
String extractZooKeeperName(const String & path)
{
static constexpr auto default_zookeeper_name = "default";
if (path.empty())
throw DB::Exception("ZooKeeper path should not be empty", DB::ErrorCodes::BAD_ARGUMENTS);
if (path[0] == '/')
return default_zookeeper_name;
auto pos = path.find(":/");
if (pos != String::npos && pos < path.find('/'))
{
auto zookeeper_name = path.substr(0, pos);
if (zookeeper_name.empty())
throw DB::Exception("Zookeeper path should start with '/' or '<auxiliary_zookeeper_name>:/'", DB::ErrorCodes::BAD_ARGUMENTS);
return zookeeper_name;
}
return default_zookeeper_name;
}
String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log)
{
if (path.empty())
throw DB::Exception("ZooKeeper path should not be empty", DB::ErrorCodes::BAD_ARGUMENTS);
if (path[0] == '/')
return normalizeZooKeeperPath(path, check_starts_with_slash, log);
auto pos = path.find(":/");
if (pos != String::npos && pos < path.find('/'))
{
return normalizeZooKeeperPath(path.substr(pos + 1, String::npos), check_starts_with_slash, log);
}
return normalizeZooKeeperPath(path, check_starts_with_slash, log);
}
}

View File

@ -379,4 +379,11 @@ private:
};
using EphemeralNodeHolderPtr = EphemeralNodeHolder::Ptr;
String normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr);
String extractZooKeeperName(const String & path);
String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr);
}

View File

@ -597,6 +597,8 @@ class IColumn;
M(Int64, input_format_orc_row_batch_size, 100'000, "Batch size when reading ORC stripes.", 0) \
M(Bool, input_format_parquet_import_nested, false, "Allow to insert array of structs into Nested table in Parquet input format.", 0) \
M(Bool, input_format_allow_seeks, true, "Allow seeks while reading in ORC/Parquet/Arrow input formats", 0) \
M(UInt64, input_format_msgpack_number_of_columns, 0, "The number of columns in inserted MsgPack data. Used for automatic schema inference from data.", 0) \
M(UInt64, input_format_max_rows_to_read_for_schema_inference, 100, "The maximum rows of data to read for automatic schema inference", 0) \
\
M(DateTimeInputFormat, date_time_input_format, FormatSettings::DateTimeInputFormat::Basic, "Method to read DateTime from text input formats. Possible values: 'basic' and 'best_effort'.", 0) \
M(DateTimeOutputFormat, date_time_output_format, FormatSettings::DateTimeOutputFormat::Simple, "Method to write DateTime to text output. Possible values: 'simple', 'iso', 'unix_timestamp'.", 0) \
@ -662,6 +664,7 @@ class IColumn;
M(Bool, output_format_arrow_low_cardinality_as_dictionary, false, "Enable output LowCardinality type as Dictionary Arrow type", 0) \
\
M(EnumComparingMode, format_capn_proto_enum_comparising_mode, FormatSettings::EnumComparingMode::BY_VALUES, "How to map ClickHouse Enum and CapnProto Enum", 0)\
// End of FORMAT_FACTORY_SETTINGS
// Please add settings non-related to formats into the COMMON_SETTINGS above.

View File

@ -377,6 +377,8 @@ struct WhichDataType
constexpr bool isNullable() const { return idx == TypeIndex::Nullable; }
constexpr bool isFunction() const { return idx == TypeIndex::Function; }
constexpr bool isAggregateFunction() const { return idx == TypeIndex::AggregateFunction; }
constexpr bool isLowCarnality() const { return idx == TypeIndex::LowCardinality; }
};
/// IDataType helpers (alternative for IDataType virtual methods with single point of truth)

View File

@ -76,11 +76,17 @@ std::pair<String, StoragePtr> createTableFromAST(
/// - the database has not been loaded yet;
/// - the code is simpler, since the query is already brought to a suitable form.
if (!ast_create_query.columns_list || !ast_create_query.columns_list->columns)
{
if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(ast_create_query.storage->engine->name))
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
/// Leave columns empty.
}
else
{
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true);
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
}
}
return
{

View File

@ -7,6 +7,8 @@
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/IDataType.h>
#include <boost/algorithm/string.hpp>
#include <boost/algorithm/string/join.hpp>
@ -26,6 +28,7 @@ namespace ErrorCodes
extern const int FILE_DOESNT_EXIST;
extern const int UNKNOWN_EXCEPTION;
extern const int INCORRECT_DATA;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
capnp::StructSchema CapnProtoSchemaParser::getMessageSchema(const FormatSchemaInfo & schema_info)
@ -427,6 +430,112 @@ void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Blo
}
}
template <typename ValueType>
static DataTypePtr getEnumDataTypeFromEnumerants(const capnp::EnumSchema::EnumerantList & enumerants)
{
std::vector<std::pair<String, ValueType>> values;
for (auto enumerant : enumerants)
values.emplace_back(enumerant.getProto().getName(), ValueType(enumerant.getOrdinal()));
return std::make_shared<DataTypeEnum<ValueType>>(std::move(values));
}
static DataTypePtr getEnumDataTypeFromEnumSchema(const capnp::EnumSchema & enum_schema)
{
auto enumerants = enum_schema.getEnumerants();
if (enumerants.size() < 128)
return getEnumDataTypeFromEnumerants<Int8>(enumerants);
if (enumerants.size() < 32768)
return getEnumDataTypeFromEnumerants<Int16>(enumerants);
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "ClickHouse supports only 8 and 16-but Enums");
}
static DataTypePtr getDataTypeFromCapnProtoType(const capnp::Type & capnp_type)
{
switch (capnp_type.which())
{
case capnp::schema::Type::INT8:
return std::make_shared<DataTypeInt8>();
case capnp::schema::Type::INT16:
return std::make_shared<DataTypeInt16>();
case capnp::schema::Type::INT32:
return std::make_shared<DataTypeInt32>();
case capnp::schema::Type::INT64:
return std::make_shared<DataTypeInt64>();
case capnp::schema::Type::BOOL: [[fallthrough]];
case capnp::schema::Type::UINT8:
return std::make_shared<DataTypeUInt8>();
case capnp::schema::Type::UINT16:
return std::make_shared<DataTypeUInt16>();
case capnp::schema::Type::UINT32:
return std::make_shared<DataTypeUInt32>();
case capnp::schema::Type::UINT64:
return std::make_shared<DataTypeUInt64>();
case capnp::schema::Type::FLOAT32:
return std::make_shared<DataTypeFloat32>();
case capnp::schema::Type::FLOAT64:
return std::make_shared<DataTypeFloat64>();
case capnp::schema::Type::DATA: [[fallthrough]];
case capnp::schema::Type::TEXT:
return std::make_shared<DataTypeString>();
case capnp::schema::Type::ENUM:
return getEnumDataTypeFromEnumSchema(capnp_type.asEnum());
case capnp::schema::Type::LIST:
{
auto list_schema = capnp_type.asList();
auto nested_type = getDataTypeFromCapnProtoType(list_schema.getElementType());
return std::make_shared<DataTypeArray>(nested_type);
}
case capnp::schema::Type::STRUCT:
{
auto struct_schema = capnp_type.asStruct();
if (checkIfStructContainsUnnamedUnion(struct_schema))
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported");
/// Check if it can be Nullable.
if (checkIfStructIsNamedUnion(struct_schema))
{
auto fields = struct_schema.getUnionFields();
if (fields.size() != 2 || (!fields[0].getType().isVoid() && !fields[1].getType().isVoid()))
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unions are not supported");
auto value_type = fields[0].getType().isVoid() ? fields[1].getType() : fields[0].getType();
if (value_type.isStruct() || value_type.isList())
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Tuples and Lists cannot be inside Nullable");
auto nested_type = getDataTypeFromCapnProtoType(value_type);
return std::make_shared<DataTypeNullable>(nested_type);
}
/// Treat Struct as Tuple.
DataTypes nested_types;
Names nested_names;
for (auto field : struct_schema.getNonUnionFields())
{
nested_names.push_back(field.getProto().getName());
nested_types.push_back(getDataTypeFromCapnProtoType(field.getType()));
}
return std::make_shared<DataTypeTuple>(std::move(nested_types), std::move(nested_names));
}
default:
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unsupported CapnProtoType: {}", getCapnProtoFullTypeName(capnp_type));
}
}
NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema)
{
if (checkIfStructContainsUnnamedUnion(schema))
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Unnamed union is not supported");
NamesAndTypesList names_and_types;
for (auto field : schema.getNonUnionFields())
{
auto name = field.getProto().getName();
auto type = getDataTypeFromCapnProtoType(field.getType());
names_and_types.emplace_back(name, type);
}
return names_and_types;
}
}
#endif

View File

@ -38,6 +38,7 @@ capnp::DynamicValue::Reader getReaderByColumnName(const capnp::DynamicStruct::Re
void checkCapnProtoSchemaStructure(const capnp::StructSchema & schema, const Block & header, FormatSettings::EnumComparingMode mode);
NamesAndTypesList capnProtoSchemaToCHSchema(const capnp::StructSchema & schema);
}
#endif

View File

@ -1,7 +1,16 @@
#include <Formats/EscapingRuleUtils.h>
#include <Formats/JSONEachRowUtils.h>
#include <Formats/ReadSchemaUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h>
#include <Poco/JSON/Parser.h>
#include <Parsers/TokenIterator.h>
#include <Parsers/ExpressionListParsers.h>
#include <Interpreters/evaluateConstantExpression.h>
namespace DB
{
@ -9,6 +18,7 @@ namespace DB
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
}
FormatSettings::EscapingRule stringToEscapingRule(const String & escaping_rule)
@ -193,30 +203,149 @@ void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSe
}
}
String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
template <bool read_string>
String readByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
{
String result;
switch (escaping_rule)
{
case FormatSettings::EscapingRule::Quoted:
if constexpr (read_string)
readQuotedString(result, buf);
else
readQuotedFieldIntoString(result, buf);
break;
case FormatSettings::EscapingRule::JSON:
if constexpr (read_string)
readJSONString(result, buf);
else
readJSONFieldIntoString(result, buf);
break;
case FormatSettings::EscapingRule::Raw:
readString(result, buf);
break;
case FormatSettings::EscapingRule::CSV:
if constexpr (read_string)
readCSVString(result, buf, format_settings.csv);
else
readCSVField(result, buf, format_settings.csv);
break;
case FormatSettings::EscapingRule::Escaped:
readEscapedString(result, buf);
break;
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read string with {} escaping rule", escapingRuleToString(escaping_rule));
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot read value with {} escaping rule", escapingRuleToString(escaping_rule));
}
return result;
}
String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
{
return readByEscapingRule<false>(buf, escaping_rule, format_settings);
}
String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
{
return readByEscapingRule<true>(buf, escaping_rule, format_settings);
}
static bool evaluateConstantExpressionFromString(const StringRef & field, DataTypePtr & type, ContextPtr context)
{
if (!context)
throw Exception(ErrorCodes::LOGICAL_ERROR, "You must provide context to evaluate constant expression");
ParserExpression parser;
Expected expected;
Tokens tokens(field.data, field.data + field.size);
IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth);
ASTPtr ast;
/// FIXME: Our parser cannot parse maps in the form of '{key : value}' that is used in text formats.
bool parsed = parser.parse(token_iterator, ast, expected);
if (!parsed)
return false;
try
{
std::pair<Field, DataTypePtr> result = evaluateConstantExpression(ast, context);
type = generalizeDataType(result.second);
return true;
}
catch (...)
{
return false;
}
}
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context)
{
switch (escaping_rule)
{
case FormatSettings::EscapingRule::Quoted:
{
DataTypePtr type;
bool parsed = evaluateConstantExpressionFromString(field, type, context);
return parsed ? type : nullptr;
}
case FormatSettings::EscapingRule::JSON:
{
Poco::JSON::Parser parser;
Poco::Dynamic::Var var = parser.parse(field);
return getDataTypeFromJSONField(var);
}
case FormatSettings::EscapingRule::CSV:
{
if (field.empty() || field == format_settings.csv.null_representation)
return nullptr;
if (field == format_settings.bool_false_representation || field == format_settings.bool_true_representation)
return std::make_shared<DataTypeUInt8>();
DataTypePtr type;
bool parsed;
if (field[0] == '\'' || field[0] == '"')
{
/// Try to evaluate expression inside quotes.
parsed = evaluateConstantExpressionFromString(StringRef(field.data() + 1, field.size() - 2), type, context);
/// If it's a number in quotes we determine it as a string.
if (parsed && type && isNumber(removeNullable(type)))
return makeNullable(std::make_shared<DataTypeString>());
}
else
parsed = evaluateConstantExpressionFromString(field, type, context);
/// If we couldn't parse an expression, determine it as a string.
return parsed ? type : makeNullable(std::make_shared<DataTypeString>());
}
case FormatSettings::EscapingRule::Raw: [[fallthrough]];
case FormatSettings::EscapingRule::Escaped:
/// TODO: Try to use some heuristics here to determine the type of data.
return field.empty() ? nullptr : makeNullable(std::make_shared<DataTypeString>());
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the type for value with {} escaping rule", escapingRuleToString(escaping_rule));
}
}
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context)
{
DataTypes data_types;
data_types.reserve(fields.size());
for (const auto & field : fields)
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, escaping_rule, context));
return data_types;
}
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule)
{
switch (escaping_rule)
{
case FormatSettings::EscapingRule::CSV: [[fallthrough]];
case FormatSettings::EscapingRule::Escaped: [[fallthrough]];
case FormatSettings::EscapingRule::Raw:
return makeNullable(std::make_shared<DataTypeString>());
default:
return nullptr;
}
}
}

View File

@ -4,6 +4,7 @@
#include <DataTypes/IDataType.h>
#include <DataTypes/Serializations/ISerialization.h>
#include <IO/ReadBuffer.h>
#include <Interpreters/Context.h>
namespace DB
{
@ -33,5 +34,24 @@ void serializeFieldByEscapingRule(
void writeStringByEscapingRule(const String & value, WriteBuffer & out, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings);
String readStringByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings);
String readFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings);
/// Try to determine the type of the field written by a specific escaping rule.
/// If cannot, return nullptr.
/// - For Quoted escaping rule we can interpret a single field as a constant
/// expression and get it's type by evaluation this expression.
/// - For JSON escaping rule we can use JSON parser to parse a single field
/// and then convert JSON type of this field to ClickHouse type.
/// - For CSV escaping rule we can do the next:
/// - If the field is an unquoted string, then we could try to evaluate it
/// as a constant expression, and if it fails, treat it as a String.
/// - If the field is a string in quotes, then we can try to evaluate
/// expression inside quotes as a constant expression, and if it fails or
/// the result is a number (we don't parse numbers in quotes) we treat it as a String.
/// - For TSV and TSVRaw we treat each field as a String (TODO: try to use some tweaks and heuristics here)
DataTypePtr determineDataTypeByEscapingRule(const String & field, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr);
DataTypes determineDataTypesByEscapingRule(const std::vector<String> & fields, const FormatSettings & format_settings, FormatSettings::EscapingRule escaping_rule, ContextPtr context = nullptr);
DataTypePtr getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule escaping_rule);
}

View File

@ -13,9 +13,6 @@
#include <Processors/Formats/Impl/ParallelFormattingOutputFormat.h>
#include <Poco/URI.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ReadHelpers.h>
namespace DB
{
@ -119,6 +116,8 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
format_settings.defaults_for_omitted_fields = settings.input_format_defaults_for_omitted_fields;
format_settings.capn_proto.enum_comparing_mode = settings.format_capn_proto_enum_comparising_mode;
format_settings.seekable_read = settings.input_format_allow_seeks;
format_settings.msgpack.number_of_columns = settings.input_format_msgpack_number_of_columns;
format_settings.max_rows_to_read_for_schema_inference = settings.input_format_max_rows_to_read_for_schema_inference;
/// Validate avro_schema_registry_url with RemoteHostFilter when non-empty and in Server context
if (format_settings.schema.is_server)
@ -200,7 +199,6 @@ InputFormatPtr FormatFactory::getInput(
return format;
}
InputFormatPtr FormatFactory::getInputFormat(
const String & name,
ReadBuffer & buf,
@ -325,6 +323,32 @@ String FormatFactory::getContentType(
return format->getContentType();
}
SchemaReaderPtr FormatFactory::getSchemaReader(
const String & name,
ReadBuffer & buf,
ContextPtr context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & schema_reader_creator = dict.at(name).schema_reader_creator;
if (!schema_reader_creator)
throw Exception("FormatFactory: Format " + name + " doesn't support schema inference.", ErrorCodes::LOGICAL_ERROR);
auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context);
return schema_reader_creator(buf, format_settings, context);
}
ExternalSchemaReaderPtr FormatFactory::getExternalSchemaReader(
const String & name,
ContextPtr context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & external_schema_reader_creator = dict.at(name).external_schema_reader_creator;
if (!external_schema_reader_creator)
throw Exception("FormatFactory: Format " + name + " doesn't support schema inference.", ErrorCodes::LOGICAL_ERROR);
auto format_settings = _format_settings ? *_format_settings : getFormatSettings(context);
return external_schema_reader_creator(format_settings);
}
void FormatFactory::registerInputFormat(const String & name, InputCreator input_creator)
{
@ -358,6 +382,21 @@ void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegm
target = std::move(file_segmentation_engine);
}
void FormatFactory::registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator)
{
auto & target = dict[name].schema_reader_creator;
if (target)
throw Exception("FormatFactory: Schema reader " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(schema_reader_creator);
}
void FormatFactory::registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator)
{
auto & target = dict[name].external_schema_reader_creator;
if (target)
throw Exception("FormatFactory: Schema reader " + name + " is already registered", ErrorCodes::LOGICAL_ERROR);
target = std::move(external_schema_reader_creator);
}
void FormatFactory::markOutputFormatSupportsParallelFormatting(const String & name)
{
@ -395,6 +434,23 @@ bool FormatFactory::isOutputFormat(const String & name) const
return it != dict.end() && it->second.output_creator;
}
bool FormatFactory::checkIfFormatHasSchemaReader(const String & name)
{
const auto & target = getCreators(name);
return bool(target.schema_reader_creator);
}
bool FormatFactory::checkIfFormatHasExternalSchemaReader(const String & name)
{
const auto & target = getCreators(name);
return bool(target.external_schema_reader_creator);
}
bool FormatFactory::checkIfFormatHasAnySchemaReader(const String & name)
{
return checkIfFormatHasSchemaReader(name) || checkIfFormatHasExternalSchemaReader(name);
}
FormatFactory & FormatFactory::instance()
{
static FormatFactory ret;

View File

@ -4,7 +4,9 @@
#include <Columns/IColumn.h>
#include <Formats/FormatSettings.h>
#include <Interpreters/Context_fwd.h>
#include <IO/BufferWithOwnMemory.h>
#include <base/types.h>
#include <Core/NamesAndTypes.h>
#include <boost/noncopyable.hpp>
@ -31,6 +33,11 @@ class IOutputFormat;
struct RowInputFormatParams;
struct RowOutputFormatParams;
class ISchemaReader;
class IExternalSchemaReader;
using SchemaReaderPtr = std::shared_ptr<ISchemaReader>;
using ExternalSchemaReaderPtr = std::shared_ptr<IExternalSchemaReader>;
using InputFormatPtr = std::shared_ptr<IInputFormat>;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
@ -85,11 +92,16 @@ private:
/// The checker should return true if parallel parsing should be disabled.
using NonTrivialPrefixAndSuffixChecker = std::function<bool(ReadBuffer & buf)>;
using SchemaReaderCreator = std::function<SchemaReaderPtr(ReadBuffer & in, const FormatSettings & settings, ContextPtr context)>;
using ExternalSchemaReaderCreator = std::function<ExternalSchemaReaderPtr(const FormatSettings & settings)>;
struct Creators
{
InputCreator input_creator;
OutputCreator output_creator;
FileSegmentationEngine file_segmentation_engine;
SchemaReaderCreator schema_reader_creator;
ExternalSchemaReaderCreator external_schema_reader_creator;
bool supports_parallel_formatting{false};
bool is_column_oriented{false};
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
@ -138,6 +150,17 @@ public:
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
SchemaReaderPtr getSchemaReader(
const String & name,
ReadBuffer & buf,
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
ExternalSchemaReaderPtr getExternalSchemaReader(
const String & name,
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker);
@ -146,11 +169,19 @@ public:
void registerInputFormat(const String & name, InputCreator input_creator);
void registerOutputFormat(const String & name, OutputCreator output_creator);
/// Register schema readers for format its name.
void registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator);
void registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator);
void markOutputFormatSupportsParallelFormatting(const String & name);
void markFormatAsColumnOriented(const String & name);
bool checkIfFormatIsColumnOriented(const String & name);
bool checkIfFormatHasSchemaReader(const String & name);
bool checkIfFormatHasExternalSchemaReader(const String & name);
bool checkIfFormatHasAnySchemaReader(const String & name);
const FormatsDictionary & getAllFormats() const
{
return dict;
@ -163,6 +194,7 @@ private:
FormatsDictionary dict;
const Creators & getCreators(const String & name) const;
};
}

View File

@ -33,6 +33,7 @@ struct FormatSettings
bool defaults_for_omitted_fields = true;
bool seekable_read = true;
UInt64 max_rows_to_read_for_schema_inference = 100;
enum class DateTimeInputFormat
{
@ -217,6 +218,11 @@ struct FormatSettings
{
EnumComparingMode enum_comparing_mode = EnumComparingMode::BY_VALUES;
} capn_proto;
struct
{
UInt64 number_of_columns = 0;
} msgpack;
};
}

View File

@ -1,7 +1,14 @@
#include <IO/ReadHelpers.h>
#include <Formats/JSONEachRowUtils.h>
#include <Formats/ReadSchemaUtils.h>
#include <IO/ReadBufferFromString.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <Poco/JSON/Parser.h>
#include <base/find_symbols.h>
@ -92,6 +99,148 @@ static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer
return {loadAtPosition(in, memory, pos), number_of_rows};
}
template <const char opening_bracket, const char closing_bracket>
static String readJSONEachRowLineIntoStringImpl(ReadBuffer & in)
{
skipWhitespaceIfAny(in);
if (in.eof())
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file");
char * pos = in.position();
if (*pos != opening_bracket)
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSONEachRow line: {} expected, {} got", opening_bracket, *in.position());
++pos;
Memory memory;
size_t balance = 1;
bool quotes = false;
while (loadAtPosition(in, memory, pos) && balance)
{
if (quotes)
{
pos = find_first_symbols<'\\', '"'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
if (*pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
++pos;
}
else if (*pos == '"')
{
++pos;
quotes = false;
}
}
else
{
pos = find_first_symbols<opening_bracket, closing_bracket, '\\', '"'>(pos, in.buffer().end());
if (pos == in.buffer().end())
continue;
else if (*pos == opening_bracket)
{
++balance;
++pos;
}
else if (*pos == closing_bracket)
{
--balance;
++pos;
}
else if (*pos == '\\')
{
++pos;
if (loadAtPosition(in, memory, pos))
++pos;
}
else if (*pos == '"')
{
quotes = true;
++pos;
}
}
}
if (balance)
throw Exception(ErrorCodes::INCORRECT_DATA, "Cannot read JSON object: unexpected end of file");
saveUpToPosition(in, memory, pos);
return String(memory.data(), memory.size());
}
DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field)
{
if (field.isEmpty())
return nullptr;
if (field.isBoolean())
return makeNullable(std::make_shared<DataTypeUInt8>());
if (field.isNumeric())
return makeNullable(std::make_shared<DataTypeFloat64>());
if (field.isString())
return makeNullable(std::make_shared<DataTypeString>());
if (field.isArray())
{
Poco::JSON::Array::Ptr array = field.extract<Poco::JSON::Array::Ptr>();
/// Return nullptr in case of empty array because we cannot determine nested type.
if (array->size() == 0)
return nullptr;
DataTypes nested_data_types;
/// If this array contains fields with different types we will treat it as Tuple.
bool is_tuple = false;
for (size_t i = 0; i != array->size(); ++i)
{
auto type = getDataTypeFromJSONField(array->get(i));
if (!type)
return nullptr;
if (!nested_data_types.empty() && type->getName() != nested_data_types.back()->getName())
is_tuple = true;
nested_data_types.push_back(std::move(type));
}
if (is_tuple)
return std::make_shared<DataTypeTuple>(nested_data_types);
return std::make_shared<DataTypeArray>(nested_data_types.back());
}
throw Exception{ErrorCodes::INCORRECT_DATA, "Unexpected JSON type {}", field.type().name()};
}
using JSONEachRowFieldExtractor = std::function<std::vector<Poco::Dynamic::Var>(const Poco::Dynamic::Var &)>;
template <const char opening_bracket, const char closing_bracket>
static DataTypes determineColumnDataTypesFromJSONEachRowDataImpl(ReadBuffer & in, bool /*json_strings*/, JSONEachRowFieldExtractor extractor)
{
Poco::JSON::Parser parser;
DataTypes data_types;
String line = readJSONEachRowLineIntoStringImpl<opening_bracket, closing_bracket>(in);
auto var = parser.parse(line);
std::vector<Poco::Dynamic::Var> fields = extractor(var);
data_types.reserve(fields.size());
for (const auto & field : fields)
data_types.push_back(getDataTypeFromJSONField(field));
/// TODO: For JSONStringsEachRow/JSONCompactStringsEach all types will be strings.
/// Should we try to parse data inside strings somehow in this case?
return data_types;
}
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
{
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size, 1);
@ -102,6 +251,46 @@ std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size, min_rows);
}
std::unordered_map<String, DataTypePtr> readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings)
{
std::vector<String> column_names;
/// {..., "<column_name>" : <value>, ...}
auto extractor = [&](const Poco::Dynamic::Var & var)
{
Poco::JSON::Object::Ptr object = var.extract<Poco::JSON::Object::Ptr>();
column_names = object->getNames();
std::vector<Poco::Dynamic::Var> fields;
for (size_t i = 0; i != object->size(); ++i)
fields.push_back(object->get(column_names[i]));
return fields;
};
auto data_types = determineColumnDataTypesFromJSONEachRowDataImpl<'{', '}'>(in, json_strings, extractor);
std::unordered_map<String, DataTypePtr> result;
for (size_t i = 0; i != column_names.size(); ++i)
result[column_names[i]] = data_types[i];
return result;
}
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings)
{
/// [..., <value>, ...]
auto extractor = [](const Poco::Dynamic::Var & var)
{
Poco::JSON::Array::Ptr array = var.extract<Poco::JSON::Array::Ptr>();
std::vector<Poco::Dynamic::Var> fields;
fields.reserve(array->size());
for (size_t i = 0; i != array->size(); ++i)
fields.push_back(array->get(i));
return fields;
};
return determineColumnDataTypesFromJSONEachRowDataImpl<'[', ']'>(in, json_strings, extractor);
}
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
{
/// For JSONEachRow we can safely skip whitespace characters

View File

@ -3,6 +3,7 @@
#include <Formats/FormatSettings.h>
#include <IO/BufferWithOwnMemory.h>
#include <IO/ReadBuffer.h>
#include <Poco/JSON/Object.h>
#include <utility>
namespace DB
@ -11,6 +12,21 @@ namespace DB
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows);
/// Convert JSON type to ClickHouse type. Make the result type always Nullable.
/// JSON array with different nested types is treated as Tuple.
/// If cannot convert (for example when field contains null), return nullptr.
DataTypePtr getDataTypeFromJSONField(const Poco::Dynamic::Var & field);
/// Read row in JSONEachRow format and try to determine type for each field.
/// Return map {column_name : type}.
/// If cannot determine the type of some field, return nullptr for it.
std::unordered_map<String, DataTypePtr> readRowAndGetNamesAndDataTypesForJSONEachRow(ReadBuffer & in, bool json_strings);
/// Read row in JSONCompactEachRow format and try to determine type for each field.
/// If cannot determine the type of some field, return nullptr for it.
DataTypes readRowAndGetDataTypesForJSONCompactEachRow(ReadBuffer & in, bool json_strings);
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
bool readFieldImpl(ReadBuffer & in, IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, const String & column_name, const FormatSettings & format_settings, bool yield_strings);

View File

@ -14,14 +14,14 @@ namespace ErrorCodes
extern const int INVALID_TEMPLATE_FORMAT;
}
ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name)
ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name, bool allow_indexes)
{
ReadBufferFromFile schema_file(schema.absoluteSchemaPath(), 4096);
String format_string;
readStringUntilEOF(format_string, schema_file);
try
{
parse(format_string, idx_by_name);
parse(format_string, idx_by_name, allow_indexes);
}
catch (DB::Exception & e)
{
@ -33,7 +33,7 @@ ParsedTemplateFormatString::ParsedTemplateFormatString(const FormatSchemaInfo &
}
void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name)
void ParsedTemplateFormatString::parse(const String & format_string, const ColumnIdxGetter & idx_by_name, bool allow_indexes)
{
enum ParserState
{
@ -100,6 +100,8 @@ void ParsedTemplateFormatString::parse(const String & format_string, const Colum
column_idx = strtoull(column_names.back().c_str(), &col_idx_end, 10);
if (col_idx_end != column_names.back().c_str() + column_names.back().size() || errno)
column_idx = idx_by_name(column_names.back());
else if (!allow_indexes)
throw Exception(ErrorCodes::INVALID_TEMPLATE_FORMAT, "Indexes instead of names are not allowed");
}
format_idx_to_column_idx.emplace_back(column_idx);
break;

View File

@ -31,9 +31,9 @@ struct ParsedTemplateFormatString
typedef std::function<std::optional<size_t>(const String &)> ColumnIdxGetter;
ParsedTemplateFormatString() = default;
ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name);
ParsedTemplateFormatString(const FormatSchemaInfo & schema, const ColumnIdxGetter & idx_by_name, bool allow_indexes = true);
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name);
void parse(const String & format_string, const ColumnIdxGetter & idx_by_name, bool allow_indexes = true);
static const char * readMayBeQuotedColumnNameInto(const char * pos, size_t size, String & s);
size_t columnsCount() const;

View File

@ -24,6 +24,7 @@
# include <DataTypes/DataTypeMap.h>
# include <DataTypes/DataTypeNullable.h>
# include <DataTypes/DataTypeTuple.h>
# include <DataTypes/DataTypeString.h>
# include <DataTypes/Serializations/SerializationDecimal.h>
# include <DataTypes/Serializations/SerializationFixedString.h>
# include <Formats/ProtobufReader.h>
@ -56,6 +57,7 @@ namespace ErrorCodes
extern const int PROTOBUF_FIELD_NOT_REPEATED;
extern const int PROTOBUF_BAD_CAST;
extern const int LOGICAL_ERROR;
extern const int BAD_ARGUMENTS;
}
namespace
@ -3230,8 +3232,98 @@ namespace
std::function<String(size_t)> get_root_desc_function;
std::shared_ptr<ProtobufSerializer *> root_serializer_ptr;
};
template <typename Type>
DataTypePtr getEnumDataType(const google::protobuf::EnumDescriptor * enum_descriptor)
{
std::vector<std::pair<String, Type>> values;
for (int i = 0; i != enum_descriptor->value_count(); ++i)
{
const auto * enum_value_descriptor = enum_descriptor->value(i);
values.emplace_back(enum_value_descriptor->name(), enum_value_descriptor->number());
}
return std::make_shared<DataTypeEnum<Type>>(std::move(values));
}
NameAndTypePair getNameAndDataTypeFromField(const google::protobuf::FieldDescriptor * field_descriptor, bool allow_repeat = true)
{
if (allow_repeat && field_descriptor->is_repeated())
{
auto name_and_type = getNameAndDataTypeFromField(field_descriptor, false);
return {name_and_type.name, std::make_shared<DataTypeArray>(name_and_type.type)};
}
switch (field_descriptor->type())
{
case FieldTypeId::TYPE_SFIXED32: [[fallthrough]];
case FieldTypeId::TYPE_SINT32: [[fallthrough]];
case FieldTypeId::TYPE_INT32:
return {field_descriptor->name(), std::make_shared<DataTypeInt32>()};
case FieldTypeId::TYPE_SFIXED64: [[fallthrough]];
case FieldTypeId::TYPE_SINT64: [[fallthrough]];
case FieldTypeId::TYPE_INT64:
return {field_descriptor->name(), std::make_shared<DataTypeInt64>()};
case FieldTypeId::TYPE_BOOL:
return {field_descriptor->name(), std::make_shared<DataTypeUInt8>()};
case FieldTypeId::TYPE_FLOAT:
return {field_descriptor->name(), std::make_shared<DataTypeFloat32>()};
case FieldTypeId::TYPE_DOUBLE:
return {field_descriptor->name(), std::make_shared<DataTypeFloat64>()};
case FieldTypeId::TYPE_UINT32: [[fallthrough]];
case FieldTypeId::TYPE_FIXED32:
return {field_descriptor->name(), std::make_shared<DataTypeUInt32>()};
case FieldTypeId::TYPE_UINT64: [[fallthrough]];
case FieldTypeId::TYPE_FIXED64:
return {field_descriptor->name(), std::make_shared<DataTypeUInt64>()};
case FieldTypeId::TYPE_BYTES: [[fallthrough]];
case FieldTypeId::TYPE_STRING:
return {field_descriptor->name(), std::make_shared<DataTypeString>()};
case FieldTypeId::TYPE_ENUM:
{
const auto * enum_descriptor = field_descriptor->enum_type();
if (enum_descriptor->value_count() == 0)
throw Exception("Empty enum field", ErrorCodes::BAD_ARGUMENTS);
int max_abs = std::abs(enum_descriptor->value(0)->number());
for (int i = 1; i != enum_descriptor->value_count(); ++i)
{
if (std::abs(enum_descriptor->value(i)->number()) > max_abs)
max_abs = std::abs(enum_descriptor->value(i)->number());
}
if (max_abs < 128)
return {field_descriptor->name(), getEnumDataType<Int8>(enum_descriptor)};
else if (max_abs < 32768)
return {field_descriptor->name(), getEnumDataType<Int16>(enum_descriptor)};
else
throw Exception("ClickHouse supports only 8-bit and 16-bit enums", ErrorCodes::BAD_ARGUMENTS);
}
case FieldTypeId::TYPE_GROUP: [[fallthrough]];
case FieldTypeId::TYPE_MESSAGE:
{
const auto * message_descriptor = field_descriptor->message_type();
if (message_descriptor->field_count() == 1)
{
const auto * nested_field_descriptor = message_descriptor->field(0);
auto nested_name_and_type = getNameAndDataTypeFromField(nested_field_descriptor);
return {field_descriptor->name() + "_" + nested_name_and_type.name, nested_name_and_type.type};
}
else
{
DataTypes nested_types;
Strings nested_names;
for (int i = 0; i != message_descriptor->field_count(); ++i)
{
auto nested_name_and_type = getNameAndDataTypeFromField(message_descriptor->field(i));
nested_types.push_back(nested_name_and_type.type);
nested_names.push_back(nested_name_and_type.name);
}
return {field_descriptor->name(), std::make_shared<DataTypeTuple>(std::move(nested_types), std::move(nested_names))};
}
}
}
__builtin_unreachable();
}
}
std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
const Strings & column_names,
@ -3254,5 +3346,14 @@ std::unique_ptr<ProtobufSerializer> ProtobufSerializer::create(
std::vector<size_t> missing_column_indices;
return ProtobufSerializerBuilder(writer).buildMessageSerializer(column_names, data_types, missing_column_indices, message_descriptor, with_length_delimiter);
}
NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor)
{
NamesAndTypesList schema;
for (int i = 0; i != message_descriptor->field_count(); ++i)
schema.push_back(getNameAndDataTypeFromField(message_descriptor->field(i)));
return schema;
}
}
#endif

View File

@ -4,6 +4,7 @@
#if USE_PROTOBUF
# include <Columns/IColumn.h>
#include <Core/NamesAndTypes.h>
namespace google::protobuf { class Descriptor; }
@ -48,5 +49,7 @@ public:
ProtobufWriter & writer);
};
NamesAndTypesList protobufSchemaToCHSchema(const google::protobuf::Descriptor * message_descriptor);
}
#endif

View File

@ -0,0 +1,112 @@
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeTuple.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <Formats/ReadSchemaUtils.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Common/assert_cast.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int BAD_ARGUMENTS;
}
ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional<FormatSettings> & format_settings, ReadBufferCreator read_buffer_creator, ContextPtr context)
{
NamesAndTypesList names_and_types;
if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name))
{
auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings);
try
{
names_and_types = external_schema_reader->readSchema();
}
catch (const DB::Exception & e)
{
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message());
}
}
else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name))
{
auto read_buf = read_buffer_creator();
if (read_buf->eof())
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file, file is empty", format_name);
auto schema_reader = FormatFactory::instance().getSchemaReader(format_name, *read_buf, context, format_settings);
try
{
names_and_types = schema_reader->readSchema();
}
catch (const DB::Exception & e)
{
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot extract table structure from {} format file. Error: {}", format_name, e.message());
}
}
else
throw Exception(ErrorCodes::BAD_ARGUMENTS, "{} file format doesn't support schema inference", format_name);
return ColumnsDescription(names_and_types);
}
DataTypePtr generalizeDataType(DataTypePtr type)
{
WhichDataType which(type);
if (which.isNothing())
return nullptr;
if (which.isNullable())
{
const auto * nullable_type = assert_cast<const DataTypeNullable *>(type.get());
return generalizeDataType(nullable_type->getNestedType());
}
if (isNumber(type))
return makeNullable(std::make_shared<DataTypeFloat64>());
if (which.isArray())
{
const auto * array_type = assert_cast<const DataTypeArray *>(type.get());
auto nested_type = generalizeDataType(array_type->getNestedType());
return nested_type ? std::make_shared<DataTypeArray>(nested_type) : nullptr;
}
if (which.isTuple())
{
const auto * tuple_type = assert_cast<const DataTypeTuple *>(type.get());
DataTypes nested_types;
for (const auto & element : tuple_type->getElements())
{
auto nested_type = generalizeDataType(element);
if (!nested_type)
return nullptr;
nested_types.push_back(nested_type);
}
return std::make_shared<DataTypeTuple>(std::move(nested_types));
}
if (which.isMap())
{
const auto * map_type = assert_cast<const DataTypeMap *>(type.get());
auto key_type = generalizeDataType(map_type->getKeyType());
auto value_type = generalizeDataType(map_type->getValueType());
return key_type && value_type ? std::make_shared<DataTypeMap>(key_type, value_type) : nullptr;
}
if (which.isLowCarnality())
{
const auto * lc_type = assert_cast<const DataTypeLowCardinality *>(type.get());
auto nested_type = generalizeDataType(lc_type->getDictionaryType());
return nested_type ? std::make_shared<DataTypeLowCardinality>(nested_type) : nullptr;
}
return makeNullable(type);
}
}

View File

@ -0,0 +1,30 @@
#pragma once
#include <Storages/ColumnsDescription.h>
#include <Formats/FormatFactory.h>
namespace DB
{
/// Try to determine the schema of the data in specifying format.
/// For formats that have an external schema reader, it will
/// use it and won't create a read buffer.
/// For formats that have a schema reader from the data,
/// read buffer will be created by the provided creator and
/// the schema will be extracted from the data.
/// If format doesn't have any schema reader or a schema reader
/// couldn't determine the schema, an exception will be thrown.
using ReadBufferCreator = std::function<std::unique_ptr<ReadBuffer>()>;
ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional<FormatSettings> & format_settings, ReadBufferCreator read_buffer_creator, ContextPtr context);
/// Convert type to the most general type:
/// - IntN, UIntN, FloatN, Decimal -> Float64
/// - Type -> Nullable(type)
/// - Array(Type) -> Array(Nullable(Type))
/// - Tuple(Type1, ..., TypeN) -> Tuple(Nullable(Type1), ..., Nullable(TypeN))
/// - Map(KeyType, ValueType) -> Map(Nullable(KeyType), Nullable(ValueType))
/// - LowCardinality(Type) -> LowCardinality(Nullable(Type))
/// If type is Nothing or one of the nested types is Nothing, return nullptr.
DataTypePtr generalizeDataType(DataTypePtr type);
}

View File

@ -81,6 +81,28 @@ void registerInputFormatCapnProto(FormatFactory & factory);
void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory);
void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factory);
void registerArrowSchemaReader(FormatFactory & factory);
void registerParquetSchemaReader(FormatFactory & factory);
void registerORCSchemaReader(FormatFactory & factory);
void registerTSVSchemaReader(FormatFactory & factory);
void registerCSVSchemaReader(FormatFactory & factory);
void registerJSONCompactEachRowSchemaReader(FormatFactory & factory);
void registerJSONEachRowSchemaReader(FormatFactory & factory);
void registerNativeSchemaReader(FormatFactory & factory);
void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory);
void registerAvroSchemaReader(FormatFactory & factory);
void registerProtobufSchemaReader(FormatFactory & factory);
void registerLineAsStringSchemaReader(FormatFactory & factory);
void registerJSONAsStringSchemaReader(FormatFactory & factory);
void registerRawBLOBSchemaReader(FormatFactory & factory);
void registerMsgPackSchemaReader(FormatFactory & factory);
void registerCapnProtoSchemaReader(FormatFactory & factory);
void registerCustomSeparatedSchemaReader(FormatFactory & factory);
void registerRegexpSchemaReader(FormatFactory & factory);
void registerTSKVSchemaReader(FormatFactory & factory);
void registerValuesSchemaReader(FormatFactory & factory);
void registerTemplateSchemaReader(FormatFactory & factory);
void registerFormats()
{
auto & factory = FormatFactory::instance();
@ -152,6 +174,28 @@ void registerFormats()
registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(factory);
registerNonTrivialPrefixAndSuffixCheckerJSONAsString(factory);
registerArrowSchemaReader(factory);
registerParquetSchemaReader(factory);
registerORCSchemaReader(factory);
registerTSVSchemaReader(factory);
registerCSVSchemaReader(factory);
registerJSONCompactEachRowSchemaReader(factory);
registerJSONEachRowSchemaReader(factory);
registerNativeSchemaReader(factory);
registerRowBinaryWithNamesAndTypesSchemaReader(factory);
registerAvroSchemaReader(factory);
registerProtobufSchemaReader(factory);
registerLineAsStringSchemaReader(factory);
registerJSONAsStringSchemaReader(factory);
registerRawBLOBSchemaReader(factory);
registerMsgPackSchemaReader(factory);
registerCapnProtoSchemaReader(factory);
registerCustomSeparatedSchemaReader(factory);
registerRegexpSchemaReader(factory);
registerTSKVSchemaReader(factory);
registerValuesSchemaReader(factory);
registerTemplateSchemaReader(factory);
}
}

View File

@ -702,6 +702,25 @@ void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & set
readCSVStringInto(s, buf, settings);
}
void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings)
{
s.clear();
bool add_quote = false;
char quote = '\'';
if (!buf.eof() && (*buf.position() == '\'' || *buf.position() == '"'))
{
quote = *buf.position();
s.push_back(quote);
add_quote = true;
}
readCSVStringInto(s, buf, settings);
if (add_quote)
s.push_back(quote);
}
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
@ -1212,6 +1231,19 @@ void skipToNextRowOrEof(PeekableReadBuffer & buf, const String & row_after_delim
}
}
// Use PeekableReadBuffer to copy field to string after parsing.
template <typename ParseFunc>
static void readParsedValueIntoString(String & s, ReadBuffer & buf, ParseFunc parse_func)
{
PeekableReadBuffer peekable_buf(buf);
peekable_buf.setCheckpoint();
parse_func(peekable_buf);
peekable_buf.makeContinuousMemoryFromCheckpointToPos();
auto * end = peekable_buf.position();
peekable_buf.rollbackToCheckpoint();
s.append(peekable_buf.position(), end);
peekable_buf.position() = end;
}
template <char opening_bracket, char closing_bracket>
static void readQuotedFieldInBrackets(String & s, ReadBuffer & buf)
@ -1266,7 +1298,11 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf)
/// - Number: integer, float, decimal.
if (*buf.position() == '\'')
readQuotedString(s, buf);
{
s.push_back('\'');
readQuotedStringInto<false>(s, buf);
s.push_back('\'');
}
else if (*buf.position() == '[')
readQuotedFieldInBrackets<'[', ']'>(s, buf);
else if (*buf.position() == '(')
@ -1290,18 +1326,19 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf)
else
{
/// It's an integer, float or decimal. They all can be parsed as float.
/// Use PeekableReadBuffer to copy field to string after parsing.
PeekableReadBuffer peekable_buf(buf);
peekable_buf.setCheckpoint();
auto parse_func = [](ReadBuffer & in)
{
Float64 tmp;
readFloatText(tmp, peekable_buf);
peekable_buf.makeContinuousMemoryFromCheckpointToPos();
auto * end = peekable_buf.position();
peekable_buf.rollbackToCheckpoint();
s.append(peekable_buf.position(), end);
peekable_buf.position() = end;
readFloatText(tmp, in);
};
readParsedValueIntoString(s, buf, parse_func);
}
}
void readJSONFieldIntoString(String & s, ReadBuffer & buf)
{
auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); };
readParsedValueIntoString(s, buf, parse_func);
}
}

View File

@ -563,6 +563,8 @@ void readStringUntilWhitespace(String & s, ReadBuffer & buf);
*/
void readCSVString(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
/// Differ from readCSVString in that it doesn't remove quotes around field if any.
void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
/// Read and append result to array of characters.
template <typename Vector>
@ -1381,4 +1383,7 @@ struct PcgDeserializer
void readQuotedFieldIntoString(String & s, ReadBuffer & buf);
void readJSONFieldIntoString(String & s, ReadBuffer & buf);
}

View File

@ -637,13 +637,14 @@ InterpreterCreateQuery::TableProperties InterpreterCreateQuery::getTableProperti
/// Table function without columns list.
auto table_function = TableFunctionFactory::instance().get(create.as_table_function, getContext());
properties.columns = table_function->getActualTableStructure(getContext());
assert(!properties.columns.empty());
}
else if (create.is_dictionary)
{
return {};
}
else
/// We can have queries like "CREATE TABLE <table> ENGINE=<engine>" if <engine>
/// supports schema inference (will determine table structure in it's constructor).
else if (!StorageFactory::instance().checkIfStorageSupportsSchemaInterface(create.storage->engine->name))
throw Exception("Incorrect CREATE query: required list of column descriptions or AS section or SELECT.", ErrorCodes::INCORRECT_QUERY);
/// Even if query has list of columns, canonicalize it (unfold Nested columns).
@ -1083,7 +1084,10 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
{
const auto & factory = TableFunctionFactory::instance();
auto table_func = factory.get(create.as_table_function, getContext());
res = table_func->execute(create.as_table_function, getContext(), create.getTable(), properties.columns);
/// In case of CREATE AS table_function() query we should use global context
/// in storage creation because there will be no query context on server startup
/// and because storage lifetime is bigger than query context lifetime.
res = table_func->execute(create.as_table_function, getContext(), create.getTable(), properties.columns, /*use_global_context=*/true);
res->renameInMemory({create.getDatabase(), create.getTable(), create.uuid});
}
else

View File

@ -359,7 +359,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
if (as_table_function)
{
if (columns_list)
if (columns_list && !columns_list->empty())
{
frame.expression_list_always_start_on_new_line = true;
settings.ostr << (settings.one_line ? " (" : "\n(");
@ -375,7 +375,7 @@ void ASTCreateQuery::formatQueryImpl(const FormatSettings & settings, FormatStat
frame.expression_list_always_start_on_new_line = true;
if (columns_list && !as_table_function)
if (columns_list && !columns_list->empty() && !as_table_function)
{
settings.ostr << (settings.one_line ? " (" : "\n(");
FormatStateStacked frame_nested = frame;

View File

@ -50,6 +50,12 @@ public:
ASTPtr clone() const override;
void formatImpl(const FormatSettings & s, FormatState & state, FormatStateStacked frame) const override;
bool empty()
{
return (!columns || columns->children.empty()) && (!indices || indices->children.empty()) && (!constraints || constraints->children.empty())
&& (!projections || projections->children.empty());
}
};

View File

@ -557,13 +557,17 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
}
}
/** Create queries without list of columns:
* - CREATE|ATTACH TABLE ... AS ...
* - CREATE|ATTACH TABLE ... ENGINE = engine
*/
else
{
storage_p.parse(pos, storage, expected);
if (!s_as.ignore(pos, expected))
return false;
/// CREATE|ATTACH TABLE ... AS ...
if (s_as.ignore(pos, expected))
{
if (!select_p.parse(pos, select, expected)) /// AS SELECT ...
{
/// ENGINE can not be specified for table functions.
@ -586,6 +590,11 @@ bool ParserCreateTableQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expe
}
}
}
else if (!storage)
{
return false;
}
}
auto comment = parseComment(pos, expected);
auto query = std::make_shared<ASTCreateQuery>();

View File

@ -361,6 +361,8 @@ protected:
* Or:
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] AS ENGINE = engine SELECT ...
*
* Or (for engines that supports schema inference):
* CREATE|ATTACH TABLE [IF NOT EXISTS] [db.]name [UUID 'uuid'] [ON CLUSTER cluster] ENGINE = engine
*/
class ParserCreateTableQuery : public IParserBase
{

View File

@ -0,0 +1,160 @@
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/ReadSchemaUtils.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
IRowSchemaReader::IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_)
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_)
{
}
NamesAndTypesList IRowSchemaReader::readSchema()
{
DataTypes data_types = readRowAndGetDataTypes();
for (size_t row = 1; row < max_rows_to_read; ++row)
{
DataTypes new_data_types = readRowAndGetDataTypes();
if (new_data_types.empty())
/// We reached eof.
break;
if (new_data_types.size() != data_types.size())
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Rows have different amount of values");
for (size_t i = 0; i != data_types.size(); ++i)
{
/// We couldn't determine the type of this column in a new row, just skip it.
if (!new_data_types[i])
continue;
/// If we couldn't determine the type of column yet, just set the new type.
if (!data_types[i])
data_types[i] = new_data_types[i];
/// If the new type and the previous type for this column are different,
/// we will use default type if we have it or throw an exception.
else if (data_types[i]->getName() != new_data_types[i]->getName())
{
if (default_type)
data_types[i] = default_type;
else
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", new_data_types[i]->getName(), i + 1, row, data_types[i]->getName());
}
}
}
/// Check that we read at list one column.
if (data_types.empty())
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot read rows from the data");
/// If column names weren't set, use default names 'column_1', 'column_2', ...
if (column_names.empty())
{
column_names.reserve(data_types.size());
for (size_t i = 0; i != data_types.size(); ++i)
column_names.push_back("column_" + std::to_string(i + 1));
}
/// If column names were set, check that the number of names match the number of types.
else if (column_names.size() != data_types.size())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The number of column names {} differs with the number of types {}", column_names.size(), data_types.size());
NamesAndTypesList result;
for (size_t i = 0; i != data_types.size(); ++i)
{
/// Check that we could determine the type of this column.
if (!data_types[i])
{
if (!default_type)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum "
"number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference",
max_rows_to_read);
data_types[i] = default_type;
}
result.emplace_back(column_names[i], data_types[i]);
}
return result;
}
IRowWithNamesSchemaReader::IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_)
: ISchemaReader(in_), max_rows_to_read(max_rows_to_read_), default_type(default_type_)
{
}
NamesAndTypesList IRowWithNamesSchemaReader::readSchema()
{
auto names_and_types = readRowAndGetNamesAndDataTypes();
for (size_t row = 1; row < max_rows_to_read; ++row)
{
auto new_names_and_types = readRowAndGetNamesAndDataTypes();
if (new_names_and_types.empty())
/// We reached eof.
break;
for (const auto & [name, new_type] : new_names_and_types)
{
auto it = names_and_types.find(name);
/// If we didn't see this column before, just add it.
if (it == names_and_types.end())
{
names_and_types[name] = new_type;
continue;
}
auto & type = it->second;
/// If we couldn't determine the type of column yet, just set the new type.
if (!type)
type = new_type;
/// If the new type and the previous type for this column are different,
/// we will use default type if we have it or throw an exception.
else if (new_type && type->getName() != new_type->getName())
{
if (default_type)
type = default_type;
else
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Automatically defined type {} for column {} in row {} differs from type defined by previous rows: {}", type->getName(), name, row, new_type->getName());
}
}
}
/// Check that we read at list one column.
if (names_and_types.empty())
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot read rows from the data");
NamesAndTypesList result;
for (auto & [name, type] : names_and_types)
{
/// Check that we could determine the type of this column.
if (!type)
{
if (!default_type)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot determine table structure by first {} rows of data, because some columns contain only Nulls. To increase the maximum "
"number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference",
max_rows_to_read);
type = default_type;
}
result.emplace_back(name, type);
}
return result;
}
}

View File

@ -0,0 +1,87 @@
#pragma once
#include <Core/NamesAndTypes.h>
#include <DataTypes/IDataType.h>
#include <Formats/FormatSettings.h>
#include <IO/ReadBuffer.h>
namespace DB
{
/// Base class for schema inference for the data in some specific format.
/// It reads some data from read buffer and try to determine the schema
/// from read data.
class ISchemaReader
{
public:
ISchemaReader(ReadBuffer & in_) : in(in_) {}
virtual NamesAndTypesList readSchema() = 0;
virtual ~ISchemaReader() = default;
protected:
ReadBuffer & in;
};
/// Base class for schema inference for formats that read data row by row.
/// It reads data row by row (up to max_rows_to_read), determines types of columns
/// for each row and compare them with types from the previous rows. If some column
/// contains values with different types in different rows, the default type will be
/// used for this column or the exception will be thrown (if default type is not set).
class IRowSchemaReader : public ISchemaReader
{
public:
IRowSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
NamesAndTypesList readSchema() override;
protected:
/// Read one row and determine types of columns in it.
/// Return types in the same order in which the values were in the row.
/// If it's impossible to determine the type for some column, return nullptr for it.
/// Return empty list if can't read more data.
virtual DataTypes readRowAndGetDataTypes() = 0;
void setColumnNames(const std::vector<String> & names) { column_names = names; }
private:
size_t max_rows_to_read;
DataTypePtr default_type;
std::vector<String> column_names;
};
/// Base class for schema inference for formats that read data row by row and each
/// row contains column names and values (ex: JSONEachRow, TSKV).
/// Differ from IRowSchemaReader in that after reading a row we get
/// a map {column_name : type} and some columns may be missed in a single row
/// (in this case we will use types from the previous rows for missed columns).
class IRowWithNamesSchemaReader : public ISchemaReader
{
public:
IRowWithNamesSchemaReader(ReadBuffer & in_, size_t max_rows_to_read_, DataTypePtr default_type_ = nullptr);
NamesAndTypesList readSchema() override;
protected:
/// Read one row and determine types of columns in it.
/// Return map {column_name : type}.
/// If it's impossible to determine the type for some column, return nullptr for it.
/// Return empty map is can't read more data.
virtual std::unordered_map<String, DataTypePtr> readRowAndGetNamesAndDataTypes() = 0;
private:
size_t max_rows_to_read;
DataTypePtr default_type;
};
/// Base class for schema inference for formats that don't need any data to
/// determine the schema: formats with constant schema (ex: JSONAsString, LineAsString)
/// and formats that use external format schema (ex: Protobuf, CapnProto).
class IExternalSchemaReader
{
public:
virtual NamesAndTypesList readSchema() = 0;
virtual ~IExternalSchemaReader() = default;
};
}

View File

@ -85,31 +85,38 @@ void ArrowBlockInputFormat::resetParser()
record_batch_current = 0;
}
void ArrowBlockInputFormat::prepareReader()
static std::shared_ptr<arrow::RecordBatchReader> createStreamReader(ReadBuffer & in)
{
std::shared_ptr<arrow::Schema> schema;
if (stream)
{
auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique<ArrowInputStreamFromReadBuffer>(*in));
auto stream_reader_status = arrow::ipc::RecordBatchStreamReader::Open(std::make_unique<ArrowInputStreamFromReadBuffer>(in));
if (!stream_reader_status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION,
"Error while opening a table: {}", stream_reader_status.status().ToString());
stream_reader = *stream_reader_status;
schema = stream_reader->schema();
return *stream_reader_status;
}
else
static std::shared_ptr<arrow::ipc::RecordBatchFileReader> createFileReader(ReadBuffer & in, const FormatSettings & format_settings, std::atomic<int> & is_stopped)
{
auto arrow_file = asArrowFile(*in, format_settings, is_stopped);
auto arrow_file = asArrowFile(in, format_settings, is_stopped);
if (is_stopped)
return;
return nullptr;
auto file_reader_status = arrow::ipc::RecordBatchFileReader::Open(std::move(arrow_file));
if (!file_reader_status.ok())
throw Exception(ErrorCodes::UNKNOWN_EXCEPTION,
"Error while opening a table: {}", file_reader_status.status().ToString());
file_reader = *file_reader_status;
schema = file_reader->schema();
return *file_reader_status;
}
void ArrowBlockInputFormat::prepareReader()
{
if (stream)
stream_reader = createStreamReader(*in);
else
{
file_reader = createFileReader(*in, format_settings, is_stopped);
if (!file_reader)
return;
}
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), "Arrow", format_settings.arrow.import_nested);
@ -122,6 +129,27 @@ void ArrowBlockInputFormat::prepareReader()
record_batch_current = 0;
}
ArrowSchemaReader::ArrowSchemaReader(ReadBuffer & in_, bool stream_, const FormatSettings & format_settings_)
: ISchemaReader(in_), stream(stream_), format_settings(format_settings_)
{
}
NamesAndTypesList ArrowSchemaReader::readSchema()
{
std::shared_ptr<arrow::Schema> schema;
if (stream)
schema = createStreamReader(in)->schema();
else
{
std::atomic<int> is_stopped = 0;
schema = createFileReader(in, format_settings, is_stopped)->schema();
}
auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, stream ? "ArrowStream" : "Arrow");
return header.getNamesAndTypesList();
}
void registerInputFormatArrow(FormatFactory & factory)
{
factory.registerInputFormat(
@ -145,6 +173,20 @@ void registerInputFormatArrow(FormatFactory & factory)
});
}
void registerArrowSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader(
"Arrow",
[](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<ArrowSchemaReader>(buf, false, settings);
});
factory.registerSchemaReader(
"ArrowStream",
[](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<ArrowSchemaReader>(buf, true, settings);
});}
}
#else
@ -154,6 +196,8 @@ class FormatFactory;
void registerInputFormatArrow(FormatFactory &)
{
}
void registerArrowSchemaReader(FormatFactory &) {}
}
#endif

View File

@ -4,6 +4,7 @@
#if USE_ARROW
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
namespace arrow { class RecordBatchReader; }
@ -51,6 +52,18 @@ private:
std::atomic<int> is_stopped{0};
};
class ArrowSchemaReader : public ISchemaReader
{
public:
ArrowSchemaReader(ReadBuffer & in_, bool stream_, const FormatSettings & format_settings_);
NamesAndTypesList readSchema() override;
private:
bool stream;
const FormatSettings format_settings;
};
}
#endif

View File

@ -328,12 +328,13 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
const std::string & column_name,
const std::string & format_name,
bool is_nullable,
std::unordered_map<String, std::shared_ptr<ColumnWithTypeAndName>> & dictionary_values)
std::unordered_map<String, std::shared_ptr<ColumnWithTypeAndName>> & dictionary_values,
bool read_ints_as_dates)
{
if (!is_nullable && arrow_column->null_count() && arrow_column->type()->id() != arrow::Type::LIST
&& arrow_column->type()->id() != arrow::Type::MAP && arrow_column->type()->id() != arrow::Type::STRUCT)
{
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values);
auto nested_column = readColumnFromArrowColumn(arrow_column, column_name, format_name, true, dictionary_values, read_ints_as_dates);
auto nullmap_column = readByteMapFromArrowColumn(arrow_column);
auto nullable_type = std::make_shared<DataTypeNullable>(std::move(nested_column.type));
auto nullable_column = ColumnNullable::create(std::move(nested_column.column), std::move(nullmap_column));
@ -358,12 +359,14 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case arrow::Type::UINT16:
{
auto column = readColumnWithNumericData<UInt16>(arrow_column, column_name);
if (read_ints_as_dates)
column.type = std::make_shared<DataTypeDate>();
return column;
}
case arrow::Type::UINT32:
{
auto column = readColumnWithNumericData<UInt32>(arrow_column, column_name);
if (read_ints_as_dates)
column.type = std::make_shared<DataTypeDateTime>();
return column;
}
@ -376,7 +379,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case arrow::Type::MAP:
{
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates);
auto offsets_column = readOffsetsFromArrowListColumn(arrow_column);
const auto * tuple_column = assert_cast<const ColumnTuple *>(nested_column.column.get());
@ -388,7 +391,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
case arrow::Type::LIST:
{
auto arrow_nested_column = getNestedArrowColumn(arrow_column);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values);
auto nested_column = readColumnFromArrowColumn(arrow_nested_column, column_name, format_name, false, dictionary_values, read_ints_as_dates);
auto offsets_column = readOffsetsFromArrowListColumn(arrow_column);
auto array_column = ColumnArray::create(std::move(nested_column.column), std::move(offsets_column));
auto array_type = std::make_shared<DataTypeArray>(nested_column.type);
@ -413,7 +416,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
for (int i = 0; i != arrow_struct_type->num_fields(); ++i)
{
auto nested_arrow_column = std::make_shared<arrow::ChunkedArray>(nested_arrow_columns[i]);
auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values);
auto element = readColumnFromArrowColumn(nested_arrow_column, arrow_struct_type->field(i)->name(), format_name, false, dictionary_values, read_ints_as_dates);
tuple_elements.emplace_back(std::move(element.column));
tuple_types.emplace_back(std::move(element.type));
tuple_names.emplace_back(std::move(element.name));
@ -436,7 +439,7 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
dict_array.emplace_back(dict_chunk.dictionary());
}
auto arrow_dict_column = std::make_shared<arrow::ChunkedArray>(dict_array);
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values);
auto dict_column = readColumnFromArrowColumn(arrow_dict_column, column_name, format_name, false, dictionary_values, read_ints_as_dates);
/// We should convert read column to ColumnUnique.
auto tmp_lc_column = DataTypeLowCardinality(dict_column.type).createColumn();
@ -483,7 +486,7 @@ static void checkStatus(const arrow::Status & status, const String & column_name
throw Exception{ErrorCodes::UNKNOWN_EXCEPTION, "Error with a {} column '{}': {}.", format_name, column_name, status.ToString()};
}
static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name)
Block ArrowColumnToCHColumn::arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name)
{
ColumnsWithTypeAndName sample_columns;
for (const auto & field : schema.fields())
@ -493,24 +496,21 @@ static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::stri
std::unique_ptr<arrow::ArrayBuilder> array_builder;
arrow::Status status = MakeBuilder(pool, field->type(), &array_builder);
checkStatus(status, field->name(), format_name);
std::shared_ptr<arrow::Array> arrow_array;
status = array_builder->Finish(&arrow_array);
checkStatus(status, field->name(), format_name);
arrow::ArrayVector array_vector = {arrow_array};
auto arrow_column = std::make_shared<arrow::ChunkedArray>(array_vector);
std::unordered_map<std::string, std::shared_ptr<ColumnWithTypeAndName>> dict_values;
ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values);
ColumnWithTypeAndName sample_column = readColumnFromArrowColumn(arrow_column, field->name(), format_name, false, dict_values, false);
sample_columns.emplace_back(std::move(sample_column));
}
return Block(std::move(sample_columns));
}
ArrowColumnToCHColumn::ArrowColumnToCHColumn(
const arrow::Schema & schema, const std::string & format_name_, bool import_nested_)
: header(arrowSchemaToCHHeader(schema, format_name_)), format_name(format_name_), import_nested(import_nested_)
{
}
ArrowColumnToCHColumn::ArrowColumnToCHColumn(
const Block & header_, const std::string & format_name_, bool import_nested_)
: header(header_), format_name(format_name_), import_nested(import_nested_)
@ -553,7 +553,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
if (!nested_tables.contains(nested_table_name))
{
std::shared_ptr<arrow::ChunkedArray> arrow_column = name_to_column_ptr[nested_table_name];
ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values)};
ColumnsWithTypeAndName cols = {readColumnFromArrowColumn(arrow_column, nested_table_name, format_name, false, dictionary_values, true)};
Block block(cols);
nested_tables[nested_table_name] = std::make_shared<Block>(Nested::flatten(block));
}
@ -573,7 +573,7 @@ void ArrowColumnToCHColumn::arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr &
if (read_from_nested)
column = nested_tables[nested_table_name]->getByName(header_column.name);
else
column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values);
column = readColumnFromArrowColumn(arrow_column, header_column.name, format_name, false, dictionary_values, true);
try
{

View File

@ -23,16 +23,14 @@ public:
ArrowColumnToCHColumn(const Block & header_, const std::string & format_name_, bool import_nested_);
/// Constructor that create header by arrow schema. It will be useful for inserting
/// data from file without knowing table structure.
ArrowColumnToCHColumn(const arrow::Schema & schema, const std::string & format_name, bool import_nested_);
void arrowTableToCHChunk(Chunk & res, std::shared_ptr<arrow::Table> & table);
void arrowColumnsToCHChunk(Chunk & res, NameToColumnPtr & name_to_column_ptr);
static Block arrowSchemaToCHHeader(const arrow::Schema & schema, const std::string & format_name);
private:
const Block header;
const Block & header;
const std::string format_name;
bool import_nested;

View File

@ -815,6 +815,92 @@ const AvroDeserializer & AvroConfluentRowInputFormat::getOrCreateDeserializer(Sc
return it->second;
}
AvroSchemaReader::AvroSchemaReader(ReadBuffer & in_, bool confluent_, const FormatSettings & format_settings_)
: ISchemaReader(in_), confluent(confluent_), format_settings(format_settings_)
{
}
NamesAndTypesList AvroSchemaReader::readSchema()
{
avro::NodePtr root_node;
if (confluent)
{
UInt32 schema_id = readConfluentSchemaId(in);
root_node = getConfluentSchemaRegistry(format_settings)->getSchema(schema_id).root();
}
else
{
auto file_reader_ptr = std::make_unique<avro::DataFileReaderBase>(std::make_unique<InputStreamReadBufferAdapter>(in));
root_node = file_reader_ptr->dataSchema().root();
}
if (root_node->type() != avro::Type::AVRO_RECORD)
throw Exception("Root schema must be a record", ErrorCodes::TYPE_MISMATCH);
NamesAndTypesList names_and_types;
for (size_t i = 0; i != root_node->leaves(); ++i)
names_and_types.emplace_back(root_node->nameAt(i), avroNodeToDataType(root_node->leafAt(i)));
return names_and_types;
}
DataTypePtr AvroSchemaReader::avroNodeToDataType(avro::NodePtr node)
{
switch (node->type())
{
case avro::Type::AVRO_INT:
return {std::make_shared<DataTypeInt32>()};
case avro::Type::AVRO_LONG:
return std::make_shared<DataTypeInt64>();
case avro::Type::AVRO_BOOL:
return std::make_shared<DataTypeUInt8>();
case avro::Type::AVRO_FLOAT:
return std::make_shared<DataTypeFloat32>();
case avro::Type::AVRO_DOUBLE:
return std::make_shared<DataTypeFloat64>();
case avro::Type::AVRO_STRING:
return std::make_shared<DataTypeString>();
case avro::Type::AVRO_BYTES:
return std::make_shared<DataTypeFloat32>();
case avro::Type::AVRO_ENUM:
{
if (node->names() < 128)
{
EnumValues<Int8>::Values values;
for (size_t i = 0; i != node->names(); ++i)
values.emplace_back(node->nameAt(i), i);
return std::make_shared<DataTypeEnum8>(std::move(values));
}
else if (node->names() < 32768)
{
EnumValues<Int16>::Values values;
for (size_t i = 0; i != node->names(); ++i)
values.emplace_back(node->nameAt(i), i);
return std::make_shared<DataTypeEnum16>(std::move(values));
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "ClickHouse supports only 8 and 16-bit Enum.");
}
case avro::Type::AVRO_FIXED:
return std::make_shared<DataTypeFixedString>(node->fixedSize());
case avro::Type::AVRO_ARRAY:
return std::make_shared<DataTypeArray>(avroNodeToDataType(node->leafAt(0)));
case avro::Type::AVRO_NULL:
return std::make_shared<DataTypeNothing>();
case avro::Type::AVRO_UNION:
if (node->leaves() == 2 && (node->leafAt(0)->type() == avro::Type::AVRO_NULL || node->leafAt(1)->type() == avro::Type::AVRO_NULL))
{
size_t nested_leaf_index = node->leafAt(0)->type() == avro::Type::AVRO_NULL ? 1 : 0;
return makeNullable(avroNodeToDataType(node->leafAt(nested_leaf_index)));
}
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro type UNION is not supported for inserting.");
case avro::Type::AVRO_SYMBOLIC:
return avroNodeToDataType(avro::resolveSymbol(node));
default:
throw Exception(ErrorCodes::ILLEGAL_COLUMN, "Avro column {} is not supported for inserting.");
}
}
void registerInputFormatAvro(FormatFactory & factory)
{
factory.registerInputFormat("Avro", [](
@ -836,6 +922,21 @@ void registerInputFormatAvro(FormatFactory & factory)
});
}
void registerAvroSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("Avro", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<AvroSchemaReader>(buf, false, settings);
});
factory.registerSchemaReader("AvroConfluent", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<AvroSchemaReader>(buf, true, settings);
});
}
}
#else
@ -846,6 +947,8 @@ class FormatFactory;
void registerInputFormatAvro(FormatFactory &)
{
}
void registerAvroSchemaReader(FormatFactory &) {}
}
#endif

View File

@ -13,6 +13,7 @@
#include <Formats/FormatSettings.h>
#include <Formats/FormatSchemaInfo.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <avro/DataFile.hh>
#include <avro/Decoder.hh>
@ -160,6 +161,20 @@ private:
FormatSettings format_settings;
};
class AvroSchemaReader : public ISchemaReader
{
public:
AvroSchemaReader(ReadBuffer & in_, bool confluent_, const FormatSettings & format_settings_);
NamesAndTypesList readSchema() override;
private:
DataTypePtr avroNodeToDataType(avro::NodePtr node);
bool confluent;
const FormatSettings format_settings;
};
}
#endif

View File

@ -5,7 +5,6 @@
#include <Formats/registerWithNamesAndTypes.h>
#include <DataTypes/DataTypeFactory.h>
namespace DB
{
@ -15,11 +14,23 @@ namespace ErrorCodes
}
BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_, with_types_, format_settings_)
: RowInputFormatWithNamesAndTypes(
std::move(header),
in_,
std::move(params_),
with_names_,
with_types_,
format_settings_,
std::make_unique<BinaryFormatReader>(in_, format_settings_))
{
}
std::vector<String> BinaryRowInputFormat::readHeaderRow()
BinaryFormatReader::BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_)
{
}
std::vector<String> BinaryFormatReader::readHeaderRow()
{
std::vector<String> fields;
String field;
@ -31,13 +42,13 @@ std::vector<String> BinaryRowInputFormat::readHeaderRow()
return fields;
}
std::vector<String> BinaryRowInputFormat::readNames()
std::vector<String> BinaryFormatReader::readNames()
{
readVarUInt(read_columns, *in);
return readHeaderRow();
}
std::vector<String> BinaryRowInputFormat::readTypes()
std::vector<String> BinaryFormatReader::readTypes()
{
auto types = readHeaderRow();
for (const auto & type_name : types)
@ -45,31 +56,37 @@ std::vector<String> BinaryRowInputFormat::readTypes()
return types;
}
bool BinaryRowInputFormat::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/)
bool BinaryFormatReader::readField(IColumn & column, const DataTypePtr & /*type*/, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & /*column_name*/)
{
serialization->deserializeBinary(column, *in);
return true;
}
void BinaryRowInputFormat::skipHeaderRow()
void BinaryFormatReader::skipHeaderRow()
{
String tmp;
for (size_t i = 0; i < read_columns; ++i)
readStringBinary(tmp, *in);
}
void BinaryRowInputFormat::skipNames()
void BinaryFormatReader::skipNames()
{
readVarUInt(read_columns, *in);
skipHeaderRow();
}
void BinaryRowInputFormat::skipTypes()
void BinaryFormatReader::skipTypes()
{
if (read_columns == 0)
{
/// It's possible only when with_names = false and with_types = true
readVarUInt(read_columns, *in);
}
skipHeaderRow();
}
void BinaryRowInputFormat::skipField(size_t file_column)
void BinaryFormatReader::skipField(size_t file_column)
{
if (file_column >= read_data_types.size())
throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, "Cannot skip unknown field in RowBinaryWithNames format, because it's type is unknown");
@ -77,6 +94,11 @@ void BinaryRowInputFormat::skipField(size_t file_column)
read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in);
}
BinaryWithNamesAndTypesSchemaReader::BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesSchemaReader(in_, 0, true, true, &reader), reader(in_, format_settings_)
{
}
void registerInputFormatRowBinary(FormatFactory & factory)
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
@ -94,4 +116,13 @@ void registerInputFormatRowBinary(FormatFactory & factory)
registerWithNamesAndTypes("RowBinary", register_func);
}
void registerRowBinaryWithNamesAndTypesSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("RowBinaryWithNamesAndTypes", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<BinaryWithNamesAndTypesSchemaReader>(buf, settings);
});
}
}

View File

@ -1,15 +1,19 @@
#pragma once
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
namespace DB
{
class ReadBuffer;
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
class ReadBuffer;
/** A stream for inputting data in a binary line-by-line format.
*/
@ -24,9 +28,15 @@ public:
/// in this format we cannot provide any DiagnosticInfo, because here we have
/// just binary data.
std::string getDiagnosticInfo() override { return {}; }
};
class BinaryFormatReader : public FormatWithNamesAndTypesReader
{
public:
BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_);
private:
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
void skipField(size_t file_column) override;
void skipNames() override;
@ -37,9 +47,24 @@ private:
std::vector<String> readTypes() override;
std::vector<String> readHeaderRow();
private:
/// Data types read from input data.
DataTypes read_data_types;
UInt64 read_columns = 0;
UInt64 read_columns;
};
class BinaryWithNamesAndTypesSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
BinaryWithNamesAndTypesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
private:
DataTypes readRowAndGetDataTypes() override
{
throw Exception{ErrorCodes::NOT_IMPLEMENTED, "Method readRowAndGetDataTypes is not implemented"};
}
BinaryFormatReader reader;
};
}

View File

@ -5,13 +5,16 @@
#include <Formats/verbosePrintString.h>
#include <Formats/registerWithNamesAndTypes.h>
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/EscapingRuleUtils.h>
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
namespace ErrorCodes
{
extern const int BAD_ARGUMENTS;
@ -26,7 +29,14 @@ CSVRowInputFormat::CSVRowInputFormat(
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_)
: RowInputFormatWithNamesAndTypes(
header_,
in_,
params_,
with_names_,
with_types_,
format_settings_,
std::make_unique<CSVFormatReader>(in_, format_settings_))
{
const String bad_delimiters = " \t\"'.UL";
if (bad_delimiters.find(format_settings.csv.delimiter) != String::npos)
@ -36,6 +46,11 @@ CSVRowInputFormat::CSVRowInputFormat(
ErrorCodes::BAD_ARGUMENTS);
}
void CSVRowInputFormat::syncAfterError()
{
skipToNextLineOrEOF(*in);
}
static void skipEndOfLine(ReadBuffer & in)
{
/// \n (Unix) or \r\n (DOS/Windows) or \n\r (Mac OS Classic)
@ -52,8 +67,10 @@ static void skipEndOfLine(ReadBuffer & in)
if (!in.eof() && *in.position() == '\n')
++in.position();
else
throw Exception("Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.", ErrorCodes::INCORRECT_DATA);
throw Exception(
"Cannot parse CSV format: found \\r (CR) not followed by \\n (LF)."
" Line must end by \\n (LF) or \\r\\n (CR LF) or \\n\\r.",
ErrorCodes::INCORRECT_DATA);
}
else if (!in.eof())
throw Exception("Expected end of line", ErrorCodes::INCORRECT_DATA);
@ -62,32 +79,38 @@ static void skipEndOfLine(ReadBuffer & in)
/// Skip `whitespace` symbols allowed in CSV.
static inline void skipWhitespacesAndTabs(ReadBuffer & in)
{
while (!in.eof()
&& (*in.position() == ' '
|| *in.position() == '\t'))
while (!in.eof() && (*in.position() == ' ' || *in.position() == '\t'))
++in.position();
}
void CSVRowInputFormat::skipFieldDelimiter()
CSVFormatReader::CSVFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_) : FormatWithNamesAndTypesReader(in_, format_settings_)
{
}
void CSVFormatReader::skipFieldDelimiter()
{
skipWhitespacesAndTabs(*in);
assertChar(format_settings.csv.delimiter, *in);
}
String CSVRowInputFormat::readFieldIntoString()
template <bool read_string>
String CSVFormatReader::readCSVFieldIntoString()
{
skipWhitespacesAndTabs(*in);
String field;
if constexpr (read_string)
readCSVString(field, *in, format_settings.csv);
else
readCSVField(field, *in, format_settings.csv);
return field;
}
void CSVRowInputFormat::skipField()
void CSVFormatReader::skipField()
{
readFieldIntoString();
readCSVFieldIntoString<true>();
}
void CSVRowInputFormat::skipRowEndDelimiter()
void CSVFormatReader::skipRowEndDelimiter()
{
skipWhitespacesAndTabs(*in);
@ -105,33 +128,32 @@ void CSVRowInputFormat::skipRowEndDelimiter()
skipEndOfLine(*in);
}
void CSVRowInputFormat::skipHeaderRow()
void CSVFormatReader::skipHeaderRow()
{
do
{
skipField();
skipWhitespacesAndTabs(*in);
}
while (checkChar(format_settings.csv.delimiter, *in));
} while (checkChar(format_settings.csv.delimiter, *in));
skipRowEndDelimiter();
}
std::vector<String> CSVRowInputFormat::readHeaderRow()
template <bool is_header>
std::vector<String> CSVFormatReader::readRowImpl()
{
std::vector<String> fields;
do
{
fields.push_back(readFieldIntoString());
fields.push_back(readCSVFieldIntoString<is_header>());
skipWhitespacesAndTabs(*in);
}
while (checkChar(format_settings.csv.delimiter, *in));
} while (checkChar(format_settings.csv.delimiter, *in));
skipRowEndDelimiter();
return fields;
}
bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
bool CSVFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
const char delimiter = format_settings.csv.delimiter;
@ -144,7 +166,8 @@ bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
if (*in->position() == '\n' || *in->position() == '\r')
{
out << "ERROR: Line feed found where delimiter (" << delimiter << ") is expected."
out << "ERROR: Line feed found where delimiter (" << delimiter
<< ") is expected."
" It's like your file has less columns than expected.\n"
"And if your file has the right number of columns, maybe it has unescaped quotes in values.\n";
}
@ -160,7 +183,7 @@ bool CSVRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
return true;
}
bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
bool CSVFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespacesAndTabs(*in);
@ -191,23 +214,21 @@ bool CSVRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
return true;
}
void CSVRowInputFormat::syncAfterError()
{
skipToNextLineOrEOF(*in);
}
bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/)
bool CSVFormatReader::readField(
IColumn & column,
const DataTypePtr & type,
const SerializationPtr & serialization,
bool is_last_file_column,
const String & /*column_name*/)
{
skipWhitespacesAndTabs(*in);
const bool at_delimiter = !in->eof() && *in->position() == format_settings.csv.delimiter;
const bool at_last_column_line_end = is_last_file_column
&& (in->eof() || *in->position() == '\n' || *in->position() == '\r');
const bool at_last_column_line_end = is_last_file_column && (in->eof() || *in->position() == '\n' || *in->position() == '\r');
/// Note: Tuples are serialized in CSV as separate columns, but with empty_as_default or null_as_default
/// only one empty or NULL column will be expected
if (format_settings.csv.empty_as_default
&& (at_delimiter || at_last_column_line_end))
if (format_settings.csv.empty_as_default && (at_delimiter || at_last_column_line_end))
{
/// Treat empty unquoted column value as default value, if
/// specified in the settings. Tuple columns might seem
@ -231,6 +252,31 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co
}
}
CSVSchemaReader::CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_)
: FormatWithNamesAndTypesSchemaReader(
in_,
format_setting_.max_rows_to_read_for_schema_inference,
with_names_,
with_types_,
&reader,
getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::CSV))
, reader(in_, format_setting_)
, context(context_)
{
}
DataTypes CSVSchemaReader::readRowAndGetDataTypes()
{
if (in.eof())
return {};
auto fields = reader.readRow();
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), FormatSettings::EscapingRule::CSV, context);
}
void registerInputFormatCSV(FormatFactory & factory)
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
@ -326,4 +372,17 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory)
registerWithNamesAndTypes("CSV", register_func);
}
void registerCSVSchemaReader(FormatFactory & factory)
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerSchemaReader(format_name, [with_names, with_types](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context)
{
return std::make_shared<CSVSchemaReader>(buf, with_names, with_types, settings, context);
});
};
registerWithNamesAndTypes("CSV", register_func);
}
}

View File

@ -5,6 +5,7 @@
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
@ -28,6 +29,12 @@ public:
private:
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
};
class CSVFormatReader : public FormatWithNamesAndTypesReader
{
public:
CSVFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_);
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
@ -48,11 +55,28 @@ private:
void skipFieldDelimiter() override;
void skipRowEndDelimiter() override;
std::vector<String> readHeaderRow();
std::vector<String> readNames() override { return readHeaderRow(); }
std::vector<String> readTypes() override { return readHeaderRow(); }
std::vector<String> readHeaderRow() { return readRowImpl<true>(); }
std::vector<String> readRow() { return readRowImpl<false>(); }
String readFieldIntoString();
template <bool is_header>
std::vector<String> readRowImpl();
template <bool read_string>
String readCSVFieldIntoString();
};
class CSVSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
CSVSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, const FormatSettings & format_setting_, ContextPtr context_);
private:
DataTypes readRowAndGetDataTypes() override;
CSVFormatReader reader;
ContextPtr context;
};
}

View File

@ -273,6 +273,7 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
#endif
auto root_reader = msg.getRoot<capnp::DynamicStruct>(root);
for (size_t i = 0; i != columns.size(); ++i)
{
auto value = getReaderByColumnName(root_reader, column_names[i]);
@ -282,6 +283,24 @@ bool CapnProtoRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
return true;
}
CapnProtoSchemaReader::CapnProtoSchemaReader(const FormatSettings & format_settings_) : format_settings(format_settings_)
{
}
NamesAndTypesList CapnProtoSchemaReader::readSchema()
{
auto schema_info = FormatSchemaInfo(
format_settings.schema.format_schema,
"CapnProto",
true,
format_settings.schema.is_server,
format_settings.schema.format_schema_path);
auto schema_parser = CapnProtoSchemaParser();
auto schema = schema_parser.getMessageSchema(schema_info);
return capnProtoSchemaToCHSchema(schema);
}
void registerInputFormatCapnProto(FormatFactory & factory)
{
factory.registerInputFormat(
@ -293,6 +312,14 @@ void registerInputFormatCapnProto(FormatFactory & factory)
});
}
void registerCapnProtoSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("CapnProto", [](const FormatSettings & settings)
{
return std::make_shared<CapnProtoSchemaReader>(settings);
});
}
}
#else
@ -301,6 +328,7 @@ namespace DB
{
class FormatFactory;
void registerInputFormatCapnProto(FormatFactory &) {}
void registerCapnProtoSchemaReader(FormatFactory &) {}
}
#endif // USE_CAPNP

View File

@ -6,6 +6,7 @@
#include <Core/Block.h>
#include <Formats/CapnProtoUtils.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
namespace DB
{
@ -38,6 +39,17 @@ private:
Names column_names;
};
class CapnProtoSchemaReader : public IExternalSchemaReader
{
public:
CapnProtoSchemaReader(const FormatSettings & format_settings_);
NamesAndTypesList readSchema() override;
private:
const FormatSettings format_settings;
};
}
#endif // USE_CAPNP

View File

@ -31,7 +31,7 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
bool ignore_spaces_,
const FormatSettings & format_settings_)
: CustomSeparatedRowInputFormat(
header_, std::make_unique<PeekableReadBuffer>(in_buf_), params_, with_names_, with_types_, ignore_spaces_, format_settings_)
header_, std::make_unique<PeekableReadBuffer>(in_buf_), params_, with_names_, with_types_, ignore_spaces_, updateFormatSettings(format_settings_))
{
}
@ -43,10 +43,15 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
bool with_types_,
bool ignore_spaces_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, *buf_, params_, with_names_, with_types_, updateFormatSettings(format_settings_))
: RowInputFormatWithNamesAndTypes(
header_,
*buf_,
params_,
with_names_,
with_types_,
format_settings_,
std::make_unique<CustomSeparatedFormatReader>(*buf_, ignore_spaces_, format_settings_))
, buf(std::move(buf_))
, ignore_spaces(ignore_spaces_)
, escaping_rule(format_settings_.custom.escaping_rule)
{
/// In case of CustomSeparatedWithNames(AndTypes) formats and enabled setting input_format_with_names_use_header we don't know
/// the exact number of columns in data (because it can contain unknown columns). So, if field_delimiter and row_after_delimiter are
@ -61,43 +66,76 @@ CustomSeparatedRowInputFormat::CustomSeparatedRowInputFormat(
}
}
void CustomSeparatedRowInputFormat::skipPrefixBeforeHeader()
bool CustomSeparatedRowInputFormat::allowSyncAfterError() const
{
return !format_settings.custom.row_after_delimiter.empty() || !format_settings.custom.row_between_delimiter.empty();
}
void CustomSeparatedRowInputFormat::syncAfterError()
{
skipToNextRowOrEof(*buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces);
end_of_stream = buf->eof();
/// It can happen that buf->position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
RowInputFormatWithNamesAndTypes::setReadBuffer(in_);
}
CustomSeparatedFormatReader::CustomSeparatedFormatReader(
PeekableReadBuffer & buf_, bool ignore_spaces_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesReader(buf_, format_settings_), buf(&buf_), ignore_spaces(ignore_spaces_)
{
}
void CustomSeparatedRowInputFormat::resetParser()
{
RowInputFormatWithNamesAndTypes::resetParser();
buf->reset();
}
void CustomSeparatedFormatReader::skipPrefixBeforeHeader()
{
skipSpaces();
assertString(format_settings.custom.result_before_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowStartDelimiter()
void CustomSeparatedFormatReader::skipRowStartDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_before_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipFieldDelimiter()
void CustomSeparatedFormatReader::skipFieldDelimiter()
{
skipSpaces();
assertString(format_settings.custom.field_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowEndDelimiter()
void CustomSeparatedFormatReader::skipRowEndDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_after_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipRowBetweenDelimiter()
void CustomSeparatedFormatReader::skipRowBetweenDelimiter()
{
skipSpaces();
assertString(format_settings.custom.row_between_delimiter, *buf);
}
void CustomSeparatedRowInputFormat::skipField()
void CustomSeparatedFormatReader::skipField()
{
skipSpaces();
skipFieldByEscapingRule(*buf, escaping_rule, format_settings);
skipFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings);
}
bool CustomSeparatedRowInputFormat::checkEndOfRow()
bool CustomSeparatedFormatReader::checkEndOfRow()
{
PeekableReadBufferCheckpoint checkpoint{*buf, true};
@ -118,43 +156,66 @@ bool CustomSeparatedRowInputFormat::checkEndOfRow()
return checkForSuffixImpl(true);
}
std::vector<String> CustomSeparatedRowInputFormat::readHeaderRow()
template <bool is_header>
String CustomSeparatedFormatReader::readFieldIntoString(bool is_first)
{
if (!is_first)
skipFieldDelimiter();
skipSpaces();
if constexpr (is_header)
return readStringByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings);
else
return readFieldByEscapingRule(*buf, format_settings.custom.escaping_rule, format_settings);
}
template <bool is_header>
std::vector<String> CustomSeparatedFormatReader::readRowImpl()
{
std::vector<String> values;
skipRowStartDelimiter();
if (columns == 0)
{
do
{
if (!values.empty())
skipFieldDelimiter();
skipSpaces();
values.push_back(readStringByEscapingRule(*buf, escaping_rule, format_settings));
values.push_back(readFieldIntoString<is_header>(values.empty()));
} while (!checkEndOfRow());
columns = values.size();
}
else
{
for (size_t i = 0; i != columns; ++i)
values.push_back(readFieldIntoString<is_header>(i == 0));
}
while (!checkEndOfRow());
skipRowEndDelimiter();
return values;
}
void CustomSeparatedRowInputFormat::skipHeaderRow()
void CustomSeparatedFormatReader::skipHeaderRow()
{
size_t columns = getPort().getHeader().columns();
skipRowStartDelimiter();
for (size_t i = 0; i != columns; ++i)
bool first = true;
do
{
skipField();
if (i + 1 != columns)
if (!first)
skipFieldDelimiter();
first = false;
skipField();
}
while (!checkEndOfRow());
skipRowEndDelimiter();
}
bool CustomSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &)
bool CustomSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool, const String &)
{
skipSpaces();
return deserializeFieldByEscapingRule(type, serialization, column, *buf, escaping_rule, format_settings);
return deserializeFieldByEscapingRule(type, serialization, column, *buf, format_settings.custom.escaping_rule, format_settings);
}
bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof)
bool CustomSeparatedFormatReader::checkForSuffixImpl(bool check_eof)
{
skipSpaces();
if (format_settings.custom.result_after_delimiter.empty())
@ -177,7 +238,7 @@ bool CustomSeparatedRowInputFormat::checkForSuffixImpl(bool check_eof)
return false;
}
bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out)
bool CustomSeparatedFormatReader::tryParseSuffixWithDiagnosticInfo(WriteBuffer & out)
{
PeekableReadBufferCheckpoint checkpoint{*buf};
if (checkForSuffixImpl(false))
@ -192,7 +253,7 @@ bool CustomSeparatedRowInputFormat::tryParseSuffixWithDiagnosticInfo(WriteBuffer
return true;
}
bool CustomSeparatedRowInputFormat::checkForSuffix()
bool CustomSeparatedFormatReader::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{*buf};
if (checkForSuffixImpl(true))
@ -201,51 +262,60 @@ bool CustomSeparatedRowInputFormat::checkForSuffix()
return false;
}
bool CustomSeparatedRowInputFormat::allowSyncAfterError() const
{
return !format_settings.custom.row_after_delimiter.empty() || !format_settings.custom.row_between_delimiter.empty();
}
void CustomSeparatedRowInputFormat::syncAfterError()
{
skipToNextRowOrEof(*buf, format_settings.custom.row_after_delimiter, format_settings.custom.row_between_delimiter, ignore_spaces);
end_of_stream = buf->eof();
/// It can happen that buf->position() is not at the beginning of row
/// if some delimiters is similar to row_format.delimiters.back() and row_between_delimiter.
/// It will cause another parsing error.
}
bool CustomSeparatedRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
bool CustomSeparatedFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_before_delimiter, "delimiter before first field", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
bool CustomSeparatedFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.field_delimiter, "delimiter between fields", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
bool CustomSeparatedFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_after_delimiter, "delimiter after last field", ignore_spaces);
}
bool CustomSeparatedRowInputFormat::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out)
bool CustomSeparatedFormatReader::parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
return parseDelimiterWithDiagnosticInfo(out, *buf, format_settings.custom.row_between_delimiter, "delimiter between rows", ignore_spaces);
}
void CustomSeparatedRowInputFormat::resetParser()
void CustomSeparatedFormatReader::setReadBuffer(ReadBuffer & in_)
{
RowInputFormatWithNamesAndTypes::resetParser();
buf->reset();
buf = assert_cast<PeekableReadBuffer *>(&in_);
FormatWithNamesAndTypesReader::setReadBuffer(in_);
}
void CustomSeparatedRowInputFormat::setReadBuffer(ReadBuffer & in_)
CustomSeparatedSchemaReader::CustomSeparatedSchemaReader(
ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_)
: FormatWithNamesAndTypesSchemaReader(
buf,
format_setting_.max_rows_to_read_for_schema_inference,
with_names_,
with_types_,
&reader,
getDefaultDataTypeForEscapingRule(format_setting_.custom.escaping_rule))
, buf(in_)
, reader(buf, ignore_spaces_, updateFormatSettings(format_setting_))
, context(context_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
}
DataTypes CustomSeparatedSchemaReader::readRowAndGetDataTypes()
{
if (reader.checkForSuffix())
return {};
if (!first_row || with_names || with_types)
reader.skipRowBetweenDelimiter();
if (first_row)
first_row = false;
auto fields = reader.readRow();
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule(), context);
}
void registerInputFormatCustomSeparated(FormatFactory & factory)
@ -267,4 +337,20 @@ void registerInputFormatCustomSeparated(FormatFactory & factory)
}
}
void registerCustomSeparatedSchemaReader(FormatFactory & factory)
{
for (bool ignore_spaces : {false, true})
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerSchemaReader(format_name, [with_names, with_types, ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context)
{
return std::make_shared<CustomSeparatedSchemaReader>(buf, with_names, with_types, ignore_spaces, settings, context);
});
};
registerWithNamesAndTypes(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", register_func);
}
}
}

View File

@ -19,7 +19,6 @@ public:
void resetParser() override;
String getName() const override { return "CustomSeparatedRowInputFormat"; }
void setReadBuffer(ReadBuffer & in_) override;
private:
@ -28,6 +27,19 @@ private:
std::unique_ptr<PeekableReadBuffer> in_buf_,
const Params & params_,
bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_settings_);
bool allowSyncAfterError() const override;
void syncAfterError() override;
std::unique_ptr<PeekableReadBuffer> buf;
bool ignore_spaces;
};
class CustomSeparatedFormatReader : public FormatWithNamesAndTypesReader
{
public:
CustomSeparatedFormatReader(PeekableReadBuffer & buf_, bool ignore_spaces_, const FormatSettings & format_settings_);
using EscapingRule = FormatSettings::EscapingRule;
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
@ -46,9 +58,6 @@ private:
bool checkForSuffix() override;
bool allowSyncAfterError() const override;
void syncAfterError() override;
bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
@ -57,15 +66,41 @@ private:
std::vector<String> readNames() override { return readHeaderRow(); }
std::vector<String> readTypes() override { return readHeaderRow(); }
std::vector<String> readHeaderRow();
std::vector<String> readHeaderRow() {return readRowImpl<true>(); }
std::vector<String> readRow() { return readRowImpl<false>(); }
bool checkEndOfRow();
bool checkForSuffixImpl(bool check_eof);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
std::unique_ptr<PeekableReadBuffer> buf;
EscapingRule getEscapingRule() { return format_settings.custom.escaping_rule; }
void setReadBuffer(ReadBuffer & in_) override;
private:
template <bool is_header>
std::vector<String> readRowImpl();
template <bool read_string>
String readFieldIntoString(bool is_first);
PeekableReadBuffer * buf;
bool ignore_spaces;
EscapingRule escaping_rule;
size_t columns = 0;
};
class CustomSeparatedSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
CustomSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool ignore_spaces_, const FormatSettings & format_setting_, ContextPtr context_);
private:
DataTypes readRowAndGetDataTypes() override;
PeekableReadBuffer buf;
CustomSeparatedFormatReader reader;
ContextPtr context;
bool first_row = true;
};
}

View File

@ -202,4 +202,12 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONAsString(FormatFactory & factor
factory.registerNonTrivialPrefixAndSuffixChecker("JSONAsString", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
}
void registerJSONAsStringSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("JSONAsString", [](const FormatSettings &)
{
return std::make_shared<JSONAsStringExternalSchemaReader>();
});
}
}

View File

@ -1,8 +1,10 @@
#pragma once
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatFactory.h>
#include <IO/PeekableReadBuffer.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
@ -39,4 +41,13 @@ private:
bool allow_new_rows = true;
};
class JSONAsStringExternalSchemaReader : public IExternalSchemaReader
{
public:
NamesAndTypesList readSchema() override
{
return {{"json", std::make_shared<DataTypeString>()}};
}
};
}

View File

@ -1,6 +1,7 @@
#include <Processors/Formats/Impl/JSONCompactEachRowRowInputFormat.h>
#include <IO/ReadHelpers.h>
#include <IO/PeekableReadBuffer.h>
#include <IO/Operators.h>
#include <Formats/FormatFactory.h>
#include <Formats/verbosePrintString.h>
@ -8,16 +9,13 @@
#include <Formats/registerWithNamesAndTypes.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <Poco/JSON/Parser.h>
namespace DB
{
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
}
JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(
const Block & header_,
ReadBuffer & in_,
@ -26,24 +24,40 @@ JSONCompactEachRowRowInputFormat::JSONCompactEachRowRowInputFormat(
bool with_types_,
bool yield_strings_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, std::move(params_), with_names_, with_types_, format_settings_)
, yield_strings(yield_strings_)
: RowInputFormatWithNamesAndTypes(
header_,
in_,
std::move(params_),
with_names_,
with_types_,
format_settings_,
std::make_unique<JSONCompactEachRowFormatReader>(in_, yield_strings_, format_settings_))
{
}
void JSONCompactEachRowRowInputFormat::skipRowStartDelimiter()
void JSONCompactEachRowRowInputFormat::syncAfterError()
{
skipToUnescapedNextLineOrEOF(*in);
}
JSONCompactEachRowFormatReader::JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesReader(in_, format_settings_), yield_strings(yield_strings_)
{
}
void JSONCompactEachRowFormatReader::skipRowStartDelimiter()
{
skipWhitespaceIfAny(*in);
assertChar('[', *in);
}
void JSONCompactEachRowRowInputFormat::skipFieldDelimiter()
void JSONCompactEachRowFormatReader::skipFieldDelimiter()
{
skipWhitespaceIfAny(*in);
assertChar(',', *in);
}
void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter()
void JSONCompactEachRowFormatReader::skipRowEndDelimiter()
{
skipWhitespaceIfAny(*in);
assertChar(']', *in);
@ -55,29 +69,18 @@ void JSONCompactEachRowRowInputFormat::skipRowEndDelimiter()
skipWhitespaceIfAny(*in);
}
String JSONCompactEachRowRowInputFormat::readFieldIntoString()
void JSONCompactEachRowFormatReader::skipField()
{
skipWhitespaceIfAny(*in);
String field;
readJSONString(field, *in);
return field;
skipJSONField(*in, "skipped_field");
}
void JSONCompactEachRowRowInputFormat::skipField(size_t file_column)
{
skipWhitespaceIfAny(*in);
skipJSONField(*in, column_mapping->names_of_columns[file_column]);
}
void JSONCompactEachRowRowInputFormat::skipHeaderRow()
void JSONCompactEachRowFormatReader::skipHeaderRow()
{
skipRowStartDelimiter();
size_t i = 0;
do
{
if (i >= column_mapping->names_of_columns.size())
throw Exception(ErrorCodes::INCORRECT_DATA, "The number of columns in a row differs from the number of column names");
skipField(i++);
skipField();
skipWhitespaceIfAny(*in);
}
while (checkChar(',', *in));
@ -85,13 +88,16 @@ void JSONCompactEachRowRowInputFormat::skipHeaderRow()
skipRowEndDelimiter();
}
std::vector<String> JSONCompactEachRowRowInputFormat::readHeaderRow()
std::vector<String> JSONCompactEachRowFormatReader::readHeaderRow()
{
skipRowStartDelimiter();
std::vector<String> fields;
String field;
do
{
fields.push_back(readFieldIntoString());
skipWhitespaceIfAny(*in);
readJSONString(field, *in);
fields.push_back(field);
skipWhitespaceIfAny(*in);
}
while (checkChar(',', *in));
@ -100,18 +106,13 @@ std::vector<String> JSONCompactEachRowRowInputFormat::readHeaderRow()
return fields;
}
bool JSONCompactEachRowRowInputFormat::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name)
bool JSONCompactEachRowFormatReader::readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool /*is_last_file_column*/, const String & column_name)
{
skipWhitespaceIfAny(*in);
return readFieldImpl(*in, column, type, serialization, column_name, format_settings, yield_strings);
}
void JSONCompactEachRowRowInputFormat::syncAfterError()
{
skipToUnescapedNextLineOrEOF(*in);
}
bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
bool JSONCompactEachRowFormatReader::parseRowStartWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespaceIfAny(*in);
if (!checkChar('[', *in))
@ -123,7 +124,7 @@ bool JSONCompactEachRowRowInputFormat::parseRowStartWithDiagnosticInfo(WriteBuff
return true;
}
bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
bool JSONCompactEachRowFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
try
{
@ -150,7 +151,7 @@ bool JSONCompactEachRowRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(Wri
return true;
}
bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
bool JSONCompactEachRowFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
skipWhitespaceIfAny(*in);
@ -180,6 +181,19 @@ bool JSONCompactEachRowRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer
return true;
}
JSONCompactEachRowRowSchemaReader::JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesSchemaReader(in_, format_settings_.max_rows_to_read_for_schema_inference, with_names_, with_types_, &reader), reader(in_, yield_strings_, format_settings_)
{
}
DataTypes JSONCompactEachRowRowSchemaReader::readRowAndGetDataTypes()
{
if (in.eof())
return {};
return readRowAndGetDataTypesForJSONCompactEachRow(in, reader.yieldStrings());
}
void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
{
for (bool yield_strings : {true, false})
@ -200,6 +214,21 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
}
}
void registerJSONCompactEachRowSchemaReader(FormatFactory & factory)
{
for (bool json_strings : {false, true})
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerSchemaReader(format_name, [=](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<JSONCompactEachRowRowSchemaReader>(buf, with_names, with_types, json_strings, settings);
});
};
registerWithNamesAndTypes(json_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
}
}
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory)
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)

View File

@ -2,6 +2,7 @@
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
@ -10,6 +11,7 @@ namespace DB
class ReadBuffer;
/** A stream for reading data in a bunch of formats:
* - JSONCompactEachRow
* - JSONCompactEachRowWithNamesAndTypes
@ -34,6 +36,13 @@ public:
private:
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
};
class JSONCompactEachRowFormatReader : public FormatWithNamesAndTypesReader
{
public:
JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_);
bool parseRowStartWithDiagnosticInfo(WriteBuffer & out) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
@ -45,7 +54,8 @@ private:
bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
void skipField(size_t file_column) override;
void skipField(size_t /*column_index*/) override { skipField(); }
void skipField();
void skipHeaderRow();
void skipNames() override { skipHeaderRow(); }
void skipTypes() override { skipHeaderRow(); }
@ -56,9 +66,21 @@ private:
std::vector<String> readHeaderRow();
std::vector<String> readNames() override { return readHeaderRow(); }
std::vector<String> readTypes() override { return readHeaderRow(); }
String readFieldIntoString();
bool yieldStrings() const { return yield_strings; }
private:
bool yield_strings;
};
class JSONCompactEachRowRowSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
JSONCompactEachRowRowSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool yield_strings_, const FormatSettings & format_settings_);
private:
DataTypes readRowAndGetDataTypes() override;
JSONCompactEachRowFormatReader reader;
};
}

View File

@ -6,6 +6,7 @@
#include <Formats/FormatFactory.h>
#include <DataTypes/NestedUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/getLeastSupertype.h>
namespace DB
{
@ -286,11 +287,7 @@ void JSONEachRowRowInputFormat::readPrefix()
skipBOMIfExists(*in);
skipWhitespaceIfAny(*in);
if (!in->eof() && *in->position() == '[')
{
++in->position();
data_in_square_brackets = true;
}
data_in_square_brackets = checkChar('[', *in);
}
void JSONEachRowRowInputFormat::readSuffix()
@ -309,6 +306,28 @@ void JSONEachRowRowInputFormat::readSuffix()
assertEOF(*in);
}
JSONEachRowSchemaReader::JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings_, const FormatSettings & format_settings)
: IRowWithNamesSchemaReader(in_, format_settings.max_rows_to_read_for_schema_inference), json_strings(json_strings_)
{
}
std::unordered_map<String, DataTypePtr> JSONEachRowSchemaReader::readRowAndGetNamesAndDataTypes()
{
if (first_row)
{
skipBOMIfExists(in);
skipWhitespaceIfAny(in);
checkChar('[', in);
first_row = false;
}
skipWhitespaceIfAny(in);
if (in.eof())
return {};
return readRowAndGetNamesAndDataTypesForJSONEachRow(in, json_strings);
}
void registerInputFormatJSONEachRow(FormatFactory & factory)
{
@ -343,4 +362,17 @@ void registerNonTrivialPrefixAndSuffixCheckerJSONEachRow(FormatFactory & factory
factory.registerNonTrivialPrefixAndSuffixChecker("JSONStringsEachRow", nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl);
}
void registerJSONEachRowSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("JSONEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_unique<JSONEachRowSchemaReader>(buf, false, settings);
});
factory.registerSchemaReader("JSONStringsEachRow", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_unique<JSONEachRowSchemaReader>(buf, true, settings);
});
}
}

View File

@ -2,6 +2,7 @@
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
@ -84,4 +85,16 @@ private:
bool yield_strings;
};
class JSONEachRowSchemaReader : public IRowWithNamesSchemaReader
{
public:
JSONEachRowSchemaReader(ReadBuffer & in_, bool json_strings, const FormatSettings & format_settings);
private:
std::unordered_map<String, DataTypePtr> readRowAndGetNamesAndDataTypes() override;
bool json_strings;
bool first_row = true;
};
}

View File

@ -72,4 +72,13 @@ void registerInputFormatLineAsString(FormatFactory & factory)
return std::make_shared<LineAsStringRowInputFormat>(sample, buf, params);
});
}
void registerLineAsStringSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("LineAsString", [](
const FormatSettings &)
{
return std::make_shared<LinaAsStringSchemaReader>();
});
}
}

View File

@ -1,7 +1,9 @@
#pragma once
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeString.h>
namespace DB
{
@ -26,4 +28,13 @@ private:
void readLineObject(IColumn & column);
};
class LinaAsStringSchemaReader : public IExternalSchemaReader
{
public:
NamesAndTypesList readSchema() override
{
return {{"line", std::make_shared<DataTypeString>()}};
}
};
}

View File

@ -11,6 +11,7 @@
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/DataTypeMap.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeString.h>
#include <Columns/ColumnArray.h>
#include <Columns/ColumnNullable.h>
@ -26,6 +27,8 @@ namespace ErrorCodes
{
extern const int ILLEGAL_COLUMN;
extern const int INCORRECT_DATA;
extern const int BAD_ARGUMENTS;
extern const int UNEXPECTED_END_OF_FILE;
}
MsgPackRowInputFormat::MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_)
@ -369,7 +372,109 @@ bool MsgPackRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
void MsgPackRowInputFormat::setReadBuffer(ReadBuffer & in_)
{
buf = std::make_unique<PeekableReadBuffer>(in_);
IInputFormat::setReadBuffer(*buf);
IInputFormat::setReadBuffer(in_);
}
MsgPackSchemaReader::MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & forma_settings_)
: IRowSchemaReader(buf, forma_settings_.max_rows_to_read_for_schema_inference), buf(in_), number_of_columns(forma_settings_.msgpack.number_of_columns)
{
if (!number_of_columns)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "You must specify setting input_format_msgpack_number_of_columns to extract table schema from MsgPack data");
}
msgpack::object_handle MsgPackSchemaReader::readObject()
{
if (buf.eof())
throw Exception(ErrorCodes::UNEXPECTED_END_OF_FILE, "Unexpected eof while parsing msgpack object");
PeekableReadBufferCheckpoint checkpoint{buf};
size_t offset = 0;
bool need_more_data = true;
msgpack::object_handle object_handle;
while (need_more_data)
{
offset = 0;
try
{
object_handle = msgpack::unpack(buf.position(), buf.buffer().end() - buf.position(), offset);
need_more_data = false;
}
catch (msgpack::insufficient_bytes &)
{
buf.position() = buf.buffer().end();
if (buf.eof())
throw Exception("Unexpected end of file while parsing msgpack object", ErrorCodes::UNEXPECTED_END_OF_FILE);
buf.position() = buf.buffer().end();
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
}
}
buf.position() += offset;
return object_handle;
}
DataTypePtr MsgPackSchemaReader::getDataType(const msgpack::object & object)
{
switch (object.type)
{
case msgpack::type::object_type::POSITIVE_INTEGER:
return makeNullable(std::make_shared<DataTypeUInt64>());
case msgpack::type::object_type::NEGATIVE_INTEGER:
return makeNullable(std::make_shared<DataTypeInt64>());
case msgpack::type::object_type::FLOAT32:
return makeNullable(std::make_shared<DataTypeFloat32>());
case msgpack::type::object_type::FLOAT64:
return makeNullable(std::make_shared<DataTypeFloat64>());
case msgpack::type::object_type::BOOLEAN:
return makeNullable(std::make_shared<DataTypeUInt8>());
case msgpack::type::object_type::BIN: [[fallthrough]];
case msgpack::type::object_type::STR:
return makeNullable(std::make_shared<DataTypeString>());
case msgpack::type::object_type::ARRAY:
{
msgpack::object_array object_array = object.via.array;
if (object_array.size)
{
auto nested_type = getDataType(object_array.ptr[0]);
if (nested_type)
return std::make_shared<DataTypeArray>(getDataType(object_array.ptr[0]));
}
return nullptr;
}
case msgpack::type::object_type::MAP:
{
msgpack::object_map object_map = object.via.map;
if (object_map.size)
{
auto key_type = getDataType(object_map.ptr[0].key);
auto value_type = getDataType(object_map.ptr[1].val);
if (key_type && value_type)
return std::make_shared<DataTypeMap>(key_type, value_type);
}
return nullptr;
}
case msgpack::type::object_type::NIL:
return nullptr;
default:
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Msgpack type is not supported");
}
}
DataTypes MsgPackSchemaReader::readRowAndGetDataTypes()
{
if (buf.eof())
return {};
DataTypes data_types;
data_types.reserve(number_of_columns);
for (size_t i = 0; i != number_of_columns; ++i)
{
auto object_handle = readObject();
data_types.push_back(getDataType(object_handle.get()));
}
return data_types;
}
void registerInputFormatMsgPack(FormatFactory & factory)
@ -384,6 +489,14 @@ void registerInputFormatMsgPack(FormatFactory & factory)
});
}
void registerMsgPackSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("MsgPack", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<MsgPackSchemaReader>(buf, settings);
});
}
}
#else
@ -394,6 +507,10 @@ class FormatFactory;
void registerInputFormatMsgPack(FormatFactory &)
{
}
void registerMsgPackSchemaReader(FormatFactory &)
{
}
}
#endif

View File

@ -6,6 +6,7 @@
#if USE_MSGPACK
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatFactory.h>
#include <IO/PeekableReadBuffer.h>
#include <msgpack.hpp>
@ -76,6 +77,20 @@ private:
const DataTypes data_types;
};
class MsgPackSchemaReader : public IRowSchemaReader
{
public:
MsgPackSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
private:
msgpack::object_handle readObject();
DataTypePtr getDataType(const msgpack::object & object);
DataTypes readRowAndGetDataTypes() override;
PeekableReadBuffer buf;
UInt64 number_of_columns;
};
}
#endif

View File

@ -1,8 +1,10 @@
#include <Formats/NativeReader.h>
#include <Formats/NativeWriter.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Processors/Transforms/AggregatingTransform.h>
@ -82,6 +84,20 @@ private:
NativeWriter writer;
};
class NativeSchemaReader : public ISchemaReader
{
public:
NativeSchemaReader(ReadBuffer & in_) : ISchemaReader(in_) {}
NamesAndTypesList readSchema() override
{
auto reader = NativeReader(in, 0);
auto block = reader.read();
return block.getNamesAndTypesList();
}
};
void registerInputFormatNative(FormatFactory & factory)
{
factory.registerInputFormat("Native", [](
@ -106,4 +122,14 @@ void registerOutputFormatNative(FormatFactory & factory)
});
}
void registerNativeSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("Native", [](ReadBuffer & buf, const FormatSettings &, ContextPtr)
{
return std::make_shared<NativeSchemaReader>(buf);
});
}
}

View File

@ -87,7 +87,8 @@ static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
return 1;
}
void ORCBlockInputFormat::prepareReader()
static void getFileReaderAndSchema(
ReadBuffer & in, std::unique_ptr<arrow::adapters::orc::ORCFileReader> & file_reader, std::shared_ptr<arrow::Schema> & schema, const FormatSettings & format_settings)
{
auto arrow_file = asArrowFile(*in, format_settings, is_stopped);
if (is_stopped)
@ -101,7 +102,13 @@ void ORCBlockInputFormat::prepareReader()
auto read_schema_result = file_reader->ReadSchema();
if (!read_schema_result.ok())
throw Exception(read_schema_result.status().ToString(), ErrorCodes::BAD_ARGUMENTS);
std::shared_ptr<arrow::Schema> schema = std::move(read_schema_result).ValueOrDie();
schema = std::move(read_schema_result).ValueOrDie();
}
void ORCBlockInputFormat::prepareReader()
{
std::shared_ptr<arrow::Schema> schema;
getFileReaderAndSchema(*in, file_reader, schema, format_settings);
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), "ORC", format_settings.orc.import_nested);
@ -128,6 +135,19 @@ void ORCBlockInputFormat::prepareReader()
}
}
ORCSchemaReader::ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_)
{
}
NamesAndTypesList ORCSchemaReader::readSchema()
{
std::unique_ptr<arrow::adapters::orc::ORCFileReader> file_reader;
std::shared_ptr<arrow::Schema> schema;
getFileReaderAndSchema(in, file_reader, schema, format_settings);
auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "ORC");
return header.getNamesAndTypesList();
}
void registerInputFormatORC(FormatFactory & factory)
{
factory.registerInputFormat(
@ -142,6 +162,17 @@ void registerInputFormatORC(FormatFactory &factory)
factory.markFormatAsColumnOriented("ORC");
}
void registerORCSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader(
"ORC",
[](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<ORCSchemaReader>(buf, settings);
}
);
}
}
#else
@ -151,6 +182,10 @@ namespace DB
void registerInputFormatORC(FormatFactory &)
{
}
void registerORCSchemaReader(FormatFactory &)
{
}
}
#endif

View File

@ -3,6 +3,7 @@
#if USE_ORC
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <arrow/adapters/orc/adapter.h>
@ -54,5 +55,16 @@ private:
std::atomic<int> is_stopped{0};
};
class ORCSchemaReader : public ISchemaReader
{
public:
ORCSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
NamesAndTypesList readSchema() override;
private:
const FormatSettings format_settings;
};
}
#endif

View File

@ -94,19 +94,30 @@ static size_t countIndicesForType(std::shared_ptr<arrow::DataType> type)
return 1;
}
static void getFileReaderAndSchema(
ReadBuffer & in,
std::unique_ptr<parquet::arrow::FileReader> & file_reader,
std::shared_ptr<arrow::Schema> & schema,
const FormatSettings & format_settings,
std::atomic<int> & is_stopped)
{
auto arrow_file = asArrowFile(in, format_settings, is_stopped);
if (is_stopped)
return;
THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader));
THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema));
}
void ParquetBlockInputFormat::prepareReader()
{
auto arrow_file = asArrowFile(*in, format_settings, is_stopped);
std::shared_ptr<arrow::Schema> schema;
getFileReaderAndSchema(*in, file_reader, schema, format_settings, is_stopped);
if (is_stopped)
return;
THROW_ARROW_NOT_OK(parquet::arrow::OpenFile(std::move(arrow_file), arrow::default_memory_pool(), &file_reader));
row_group_total = file_reader->num_row_groups();
row_group_current = 0;
std::shared_ptr<arrow::Schema> schema;
THROW_ARROW_NOT_OK(file_reader->GetSchema(&schema));
arrow_column_to_ch_column = std::make_unique<ArrowColumnToCHColumn>(getPort().getHeader(), "Parquet", format_settings.parquet.import_nested);
std::unordered_set<String> nested_table_names;
@ -130,6 +141,20 @@ void ParquetBlockInputFormat::prepareReader()
}
}
ParquetSchemaReader::ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_) : ISchemaReader(in_), format_settings(format_settings_)
{
}
NamesAndTypesList ParquetSchemaReader::readSchema()
{
std::unique_ptr<parquet::arrow::FileReader> file_reader;
std::shared_ptr<arrow::Schema> schema;
std::atomic<int> is_stopped = 0;
getFileReaderAndSchema(in, file_reader, schema, format_settings, is_stopped);
auto header = ArrowColumnToCHColumn::arrowSchemaToCHHeader(*schema, "Parquet");
return header.getNamesAndTypesList();
}
void registerInputFormatParquet(FormatFactory & factory)
{
factory.registerInputFormat(
@ -144,6 +169,17 @@ void registerInputFormatParquet(FormatFactory &factory)
factory.markFormatAsColumnOriented("Parquet");
}
void registerParquetSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader(
"Parquet",
[](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<ParquetSchemaReader>(buf, settings);
}
);
}
}
#else
@ -154,6 +190,8 @@ class FormatFactory;
void registerInputFormatParquet(FormatFactory &)
{
}
void registerParquetSchemaReader(FormatFactory &) {}
}
#endif

View File

@ -3,6 +3,7 @@
#if USE_PARQUET
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
namespace parquet::arrow { class FileReader; }
@ -44,6 +45,17 @@ private:
std::atomic<int> is_stopped{0};
};
class ParquetSchemaReader : public ISchemaReader
{
public:
ParquetSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
NamesAndTypesList readSchema() override;
private:
const FormatSettings format_settings;
};
}
#endif

View File

@ -73,6 +73,34 @@ void registerInputFormatProtobuf(FormatFactory & factory)
}
}
ProtobufSchemaReader::ProtobufSchemaReader(const FormatSettings & format_settings)
: schema_info(
format_settings.schema.format_schema,
"Protobuf",
true,
format_settings.schema.is_server,
format_settings.schema.format_schema_path)
{
}
NamesAndTypesList ProtobufSchemaReader::readSchema()
{
const auto * message_descriptor = ProtobufSchemas::instance().getMessageTypeForFormatSchema(schema_info);
return protobufSchemaToCHSchema(message_descriptor);
}
void registerProtobufSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("Protobuf", [](const FormatSettings & settings)
{
return std::make_shared<ProtobufSchemaReader>(settings);
});
factory.registerExternalSchemaReader("ProtobufSingle", [](const FormatSettings & settings)
{
return std::make_shared<ProtobufSchemaReader>(settings);
});
}
}
#else
@ -81,6 +109,8 @@ namespace DB
{
class FormatFactory;
void registerInputFormatProtobuf(FormatFactory &) {}
void registerProtobufSchemaReader(FormatFactory &) {}
}
#endif

View File

@ -3,7 +3,9 @@
#include "config_formats.h"
#if USE_PROTOBUF
# include <Formats/FormatSchemaInfo.h>
# include <Processors/Formats/IRowInputFormat.h>
# include <Processors/Formats/ISchemaReader.h>
namespace DB
{
@ -42,5 +44,16 @@ private:
std::unique_ptr<ProtobufSerializer> serializer;
};
class ProtobufSchemaReader : public IExternalSchemaReader
{
public:
ProtobufSchemaReader(const FormatSettings & format_settings);
NamesAndTypesList readSchema() override;
private:
FormatSchemaInfo schema_info;
};
}
#endif

View File

@ -51,5 +51,14 @@ void registerInputFormatRawBLOB(FormatFactory & factory)
});
}
void registerRawBLOBSchemaReader(FormatFactory & factory)
{
factory.registerExternalSchemaReader("RawBLOB", [](
const FormatSettings &)
{
return std::make_shared<RawBLOBSchemaReader>();
});
}
}

View File

@ -1,6 +1,8 @@
#pragma once
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <DataTypes/DataTypeString.h>
namespace DB
@ -22,5 +24,14 @@ private:
bool readRow(MutableColumns & columns, RowReadExtension &) override;
};
class RawBLOBSchemaReader: public IExternalSchemaReader
{
public:
NamesAndTypesList readSchema() override
{
return {{"raw_blob", std::make_shared<DataTypeString>()}};
}
};
}

View File

@ -14,18 +14,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
}
RegexpRowInputFormat::RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: RegexpRowInputFormat(std::make_unique<PeekableReadBuffer>(in_), header_, params_, format_settings_)
{
}
RegexpRowInputFormat::RegexpRowInputFormat(
std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, *buf_, std::move(params_))
, buf(std::move(buf_))
, format_settings(format_settings_)
, escaping_rule(format_settings_.regexp.escaping_rule)
, regexp(format_settings_.regexp.regexp)
RegexpFieldExtractor::RegexpFieldExtractor(const FormatSettings & format_settings) : regexp(format_settings.regexp.regexp), skip_unmatched(format_settings.regexp.skip_unmatched)
{
size_t fields_count = regexp.NumberOfCapturingGroups();
matched_fields.resize(fields_count);
@ -40,6 +29,50 @@ RegexpRowInputFormat::RegexpRowInputFormat(
}
}
bool RegexpFieldExtractor::parseRow(PeekableReadBuffer & buf)
{
PeekableReadBufferCheckpoint checkpoint{buf};
size_t line_size = 0;
do
{
char * pos = find_first_symbols<'\n', '\r'>(buf.position(), buf.buffer().end());
line_size += pos - buf.position();
buf.position() = pos;
} while (buf.position() == buf.buffer().end() && !buf.eof());
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
bool match = RE2::FullMatchN(re2::StringPiece(buf.position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
if (!match && !skip_unmatched)
throw Exception("Line \"" + std::string(buf.position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
buf.position() += line_size;
checkChar('\r', buf);
if (!buf.eof() && !checkChar('\n', buf))
throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA);
return match;
}
RegexpRowInputFormat::RegexpRowInputFormat(
ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: RegexpRowInputFormat(std::make_unique<PeekableReadBuffer>(in_), header_, params_, format_settings_)
{
}
RegexpRowInputFormat::RegexpRowInputFormat(
std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_)
: IRowInputFormat(header_, *buf_, std::move(params_))
, buf(std::move(buf_))
, format_settings(format_settings_)
, escaping_rule(format_settings_.regexp.escaping_rule)
, field_extractor(RegexpFieldExtractor(format_settings_))
{
}
void RegexpRowInputFormat::resetParser()
{
@ -50,7 +83,8 @@ void RegexpRowInputFormat::resetParser()
bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
{
const auto & type = getPort().getHeader().getByPosition(index).type;
ReadBuffer field_buf(const_cast<char *>(matched_fields[index].data()), matched_fields[index].size(), 0);
auto matched_field = field_extractor.getField(index);
ReadBuffer field_buf(const_cast<char *>(matched_field.data()), matched_field.size(), 0);
try
{
return deserializeFieldByEscapingRule(type, serializations[index], *columns[index], field_buf, escaping_rule, format_settings);
@ -64,7 +98,7 @@ bool RegexpRowInputFormat::readField(size_t index, MutableColumns & columns)
void RegexpRowInputFormat::readFieldsFromMatch(MutableColumns & columns, RowReadExtension & ext)
{
if (matched_fields.size() != columns.size())
if (field_extractor.getMatchedFieldsSize() != columns.size())
throw Exception("The number of matched fields in line doesn't match the number of columns.", ErrorCodes::INCORRECT_DATA);
ext.read_columns.assign(columns.size(), false);
@ -79,39 +113,8 @@ bool RegexpRowInputFormat::readRow(MutableColumns & columns, RowReadExtension &
if (buf->eof())
return false;
PeekableReadBufferCheckpoint checkpoint{*buf};
size_t line_size = 0;
do
{
char * pos = find_first_symbols<'\n', '\r'>(buf->position(), buf->buffer().end());
line_size += pos - buf->position();
buf->position() = pos;
} while (buf->position() == buf->buffer().end() && !buf->eof());
buf->makeContinuousMemoryFromCheckpointToPos();
buf->rollbackToCheckpoint();
bool match = RE2::FullMatchN(re2::StringPiece(buf->position(), line_size), regexp, re2_arguments_ptrs.data(), re2_arguments_ptrs.size());
bool read_line = true;
if (!match)
{
if (!format_settings.regexp.skip_unmatched)
throw Exception("Line \"" + std::string(buf->position(), line_size) + "\" doesn't match the regexp.", ErrorCodes::INCORRECT_DATA);
read_line = false;
}
if (read_line)
if (field_extractor.parseRow(*buf))
readFieldsFromMatch(columns, ext);
buf->position() += line_size;
checkChar('\r', *buf);
if (!buf->eof() && !checkChar('\n', *buf))
throw Exception("No \\n after \\r at the end of line.", ErrorCodes::INCORRECT_DATA);
return true;
}
@ -121,6 +124,35 @@ void RegexpRowInputFormat::setReadBuffer(ReadBuffer & in_)
IInputFormat::setReadBuffer(*buf);
}
RegexpSchemaReader::RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowSchemaReader(
buf,
format_settings_.max_rows_to_read_for_schema_inference,
getDefaultDataTypeForEscapingRule(format_settings_.regexp.escaping_rule))
, format_settings(format_settings_)
, field_extractor(format_settings)
, buf(in_)
{
}
DataTypes RegexpSchemaReader::readRowAndGetDataTypes()
{
if (buf.eof())
return {};
field_extractor.parseRow(buf);
DataTypes data_types;
data_types.reserve(field_extractor.getMatchedFieldsSize());
for (size_t i = 0; i != field_extractor.getMatchedFieldsSize(); ++i)
{
String field(field_extractor.getField(i));
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, format_settings.regexp.escaping_rule));
}
return data_types;
}
void registerInputFormatRegexp(FormatFactory & factory)
{
factory.registerInputFormat("Regexp", [](
@ -172,4 +204,12 @@ void registerFileSegmentationEngineRegexp(FormatFactory & factory)
factory.registerFileSegmentationEngine("Regexp", &fileSegmentationEngineRegexpImpl);
}
void registerRegexpSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("Regexp", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<RegexpSchemaReader>(buf, settings);
});
}
}

View File

@ -6,6 +6,7 @@
#include <vector>
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
#include <IO/PeekableReadBuffer.h>
@ -16,6 +17,29 @@ namespace DB
class ReadBuffer;
/// Class for extracting row fields from data by regexp.
class RegexpFieldExtractor
{
public:
RegexpFieldExtractor(const FormatSettings & format_settings);
/// Return true if row was successfully parsed and row fields were extracted.
bool parseRow(PeekableReadBuffer & buf);
re2::StringPiece getField(size_t index) { return matched_fields[index]; }
size_t getMatchedFieldsSize() const { return matched_fields.size(); }
size_t getNumberOfGroups() const { return regexp.NumberOfCapturingGroups(); }
private:
const RE2 regexp;
// The vector of fields extracted from line using regexp.
std::vector<re2::StringPiece> matched_fields;
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
std::vector<RE2::Arg> re2_arguments;
std::vector<RE2::Arg *> re2_arguments_ptrs;
bool skip_unmatched;
};
/// Regexp input format.
/// This format applies regular expression from format_regexp setting for every line of file
/// (the lines must be separated by newline character ('\n') or DOS-style newline ("\r\n")).
@ -25,7 +49,6 @@ class ReadBuffer;
class RegexpRowInputFormat : public IRowInputFormat
{
using EscapingRule = FormatSettings::EscapingRule;
public:
RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_);
@ -36,6 +59,8 @@ public:
private:
RegexpRowInputFormat(std::unique_ptr<PeekableReadBuffer> buf_, const Block & header_, Params params_, const FormatSettings & format_settings_);
using EscapingRule = FormatSettings::EscapingRule;
bool readRow(MutableColumns & columns, RowReadExtension & ext) override;
bool readField(size_t index, MutableColumns & columns);
@ -44,13 +69,21 @@ private:
std::unique_ptr<PeekableReadBuffer> buf;
const FormatSettings format_settings;
const EscapingRule escaping_rule;
RegexpFieldExtractor field_extractor;
};
const RE2 regexp;
// The vector of fields extracted from line using regexp.
std::vector<re2::StringPiece> matched_fields;
// These two vectors are needed to use RE2::FullMatchN (function for extracting fields).
std::vector<RE2::Arg> re2_arguments;
std::vector<RE2::Arg *> re2_arguments_ptrs;
class RegexpSchemaReader : public IRowSchemaReader
{
public:
RegexpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings);
private:
DataTypes readRowAndGetDataTypes() override;
using EscapingRule = FormatSettings::EscapingRule;
const FormatSettings format_settings;
RegexpFieldExtractor field_extractor;
PeekableReadBuffer buf;
};
}

View File

@ -1,7 +1,10 @@
#include <IO/ReadHelpers.h>
#include <Processors/Formats/Impl/TSKVRowInputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/EscapingRuleUtils.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <DataTypes/DataTypeString.h>
#include <DataTypes/DataTypeNullable.h>
namespace DB
@ -211,6 +214,59 @@ void TSKVRowInputFormat::resetParser()
name_buf.clear();
}
TSKVSchemaReader::TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: IRowWithNamesSchemaReader(
in_,
format_settings_.max_rows_to_read_for_schema_inference,
getDefaultDataTypeForEscapingRule(FormatSettings::EscapingRule::Escaped))
, format_settings(format_settings_)
{
}
std::unordered_map<String, DataTypePtr> TSKVSchemaReader::readRowAndGetNamesAndDataTypes()
{
if (first_row)
{
skipBOMIfExists(in);
first_row = false;
}
if (in.eof())
return {};
if (*in.position() == '\n')
{
++in.position();
return {};
}
std::unordered_map<String, DataTypePtr> names_and_types;
StringRef name_ref;
String name_tmp;
String value;
do
{
bool has_value = readName(in, name_ref, name_tmp);
if (has_value)
{
readEscapedString(value, in);
names_and_types[String(name_ref)] = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Escaped);
}
else
{
/// The only thing that can go without value is `tskv` fragment that is ignored.
if (!(name_ref.size == 4 && 0 == memcmp(name_ref.data, "tskv", 4)))
throw Exception("Found field without value while parsing TSKV format: " + name_ref.toString(), ErrorCodes::INCORRECT_DATA);
}
}
while (checkChar('\t', in));
assertChar('\n', in);
return names_and_types;
}
void registerInputFormatTSKV(FormatFactory & factory)
{
factory.registerInputFormat("TSKV", [](
@ -222,5 +278,12 @@ void registerInputFormatTSKV(FormatFactory & factory)
return std::make_shared<TSKVRowInputFormat>(buf, sample, std::move(params), settings);
});
}
void registerTSKVSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("TSKV", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<TSKVSchemaReader>(buf, settings);
});
}
}

View File

@ -2,6 +2,7 @@
#include <Core/Block.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Common/HashTable/HashMap.h>
@ -52,4 +53,16 @@ private:
/// for row like ..., non-nullable column name=\N, ...
};
class TSKVSchemaReader : public IRowWithNamesSchemaReader
{
public:
TSKVSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
private:
std::unordered_map<String, DataTypePtr> readRowAndGetNamesAndDataTypes() override;
const FormatSettings format_settings;
bool first_row = true;
};
}

View File

@ -1,13 +1,15 @@
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
#include <IO/BufferWithOwnMemory.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
#include <Formats/verbosePrintString.h>
#include <Formats/FormatFactory.h>
#include <Formats/registerWithNamesAndTypes.h>
#include <DataTypes/DataTypeLowCardinality.h>
#include <DataTypes/DataTypeFactory.h>
#include <DataTypes/DataTypeNullable.h>
#include <DataTypes/Serializations/SerializationNullable.h>
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/registerWithNamesAndTypes.h>
#include <Formats/verbosePrintString.h>
#include <Formats/EscapingRuleUtils.h>
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
namespace DB
{
@ -38,40 +40,50 @@ TabSeparatedRowInputFormat::TabSeparatedRowInputFormat(
bool with_types_,
bool is_raw_,
const FormatSettings & format_settings_)
: RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_), is_raw(is_raw_)
: RowInputFormatWithNamesAndTypes(header_, in_, params_, with_names_, with_types_, format_settings_, std::make_unique<TabSeparatedFormatReader>(in_, format_settings_, is_raw_))
{
}
void TabSeparatedRowInputFormat::skipFieldDelimiter()
TabSeparatedFormatReader::TabSeparatedFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_, bool is_raw_)
: FormatWithNamesAndTypesReader(in_, format_settings_), is_raw(is_raw_)
{
}
void TabSeparatedFormatReader::skipFieldDelimiter()
{
assertChar('\t', *in);
}
void TabSeparatedRowInputFormat::skipRowEndDelimiter()
void TabSeparatedFormatReader::skipRowEndDelimiter()
{
if (in->eof())
return;
if (unlikely(row_num <= 1))
if (unlikely(first_row))
{
checkForCarriageReturn(*in);
first_row = false;
}
assertChar('\n', *in);
}
String TabSeparatedRowInputFormat::readFieldIntoString()
String TabSeparatedFormatReader::readFieldIntoString()
{
String field;
if (is_raw)
readString(field, *in);
else
readEscapedString(field, *in);
return field;
}
void TabSeparatedRowInputFormat::skipField()
void TabSeparatedFormatReader::skipField()
{
NullOutput null_sink;
readEscapedStringInto(null_sink, *in);
readFieldIntoString();
}
void TabSeparatedRowInputFormat::skipHeaderRow()
void TabSeparatedFormatReader::skipHeaderRow()
{
do
{
@ -82,7 +94,7 @@ void TabSeparatedRowInputFormat::skipHeaderRow()
skipRowEndDelimiter();
}
std::vector<String> TabSeparatedRowInputFormat::readHeaderRow()
std::vector<String> TabSeparatedFormatReader::readRow()
{
std::vector<String> fields;
do
@ -95,7 +107,7 @@ std::vector<String> TabSeparatedRowInputFormat::readHeaderRow()
return fields;
}
bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr & type,
bool TabSeparatedFormatReader::readField(IColumn & column, const DataTypePtr & type,
const SerializationPtr & serialization, bool is_last_file_column, const String & /*column_name*/)
{
const bool at_delimiter = !is_last_file_column && !in->eof() && *in->position() == '\t';
@ -118,6 +130,7 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr &
return true;
}
if (as_nullable)
return SerializationNullable::deserializeTextEscapedImpl(column, *in, format_settings, serialization);
@ -125,7 +138,7 @@ bool TabSeparatedRowInputFormat::readField(IColumn & column, const DataTypePtr &
return true;
}
bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
bool TabSeparatedFormatReader::parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out)
{
try
{
@ -156,7 +169,7 @@ bool TabSeparatedRowInputFormat::parseFieldDelimiterWithDiagnosticInfo(WriteBuff
return true;
}
bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
bool TabSeparatedFormatReader::parseRowEndWithDiagnosticInfo(WriteBuffer & out)
{
if (in->eof())
return true;
@ -190,7 +203,7 @@ bool TabSeparatedRowInputFormat::parseRowEndWithDiagnosticInfo(WriteBuffer & out
return true;
}
void TabSeparatedRowInputFormat::checkNullValueForNonNullable(DataTypePtr type)
void TabSeparatedFormatReader::checkNullValueForNonNullable(DataTypePtr type)
{
bool can_be_parsed_as_null = type->isNullable() || type->isLowCardinalityNullable() || format_settings.null_as_default;
@ -218,6 +231,28 @@ void TabSeparatedRowInputFormat::syncAfterError()
skipToUnescapedNextLineOrEOF(*in);
}
TabSeparatedSchemaReader::TabSeparatedSchemaReader(
ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings_)
: FormatWithNamesAndTypesSchemaReader(
in_,
format_settings_.max_rows_to_read_for_schema_inference,
with_names_,
with_types_,
&reader,
getDefaultDataTypeForEscapingRule(is_raw_ ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped))
, reader(in_, format_settings_, is_raw_)
{
}
DataTypes TabSeparatedSchemaReader::readRowAndGetDataTypes()
{
if (in.eof())
return {};
auto fields = reader.readRow();
return determineDataTypesByEscapingRule(fields, reader.getFormatSettings(), reader.getEscapingRule());
}
void registerInputFormatTabSeparated(FormatFactory & factory)
{
for (bool is_raw : {false, true})
@ -239,6 +274,23 @@ void registerInputFormatTabSeparated(FormatFactory & factory)
}
}
void registerTSVSchemaReader(FormatFactory & factory)
{
for (bool is_raw : {false, true})
{
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
{
factory.registerSchemaReader(format_name, [with_names, with_types, is_raw](ReadBuffer & buf, const FormatSettings & settings, ContextPtr)
{
return std::make_shared<TabSeparatedSchemaReader>(buf, with_names, with_types, is_raw, settings);
});
};
registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
}
}
static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, bool is_raw, size_t min_rows)
{
bool need_more_data = true;

View File

@ -3,6 +3,7 @@
#include <Core/Block.h>
#include <Formats/FormatSettings.h>
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
namespace DB
@ -24,6 +25,13 @@ public:
private:
bool allowSyncAfterError() const override { return true; }
void syncAfterError() override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
};
class TabSeparatedFormatReader : public FormatWithNamesAndTypesReader
{
public:
TabSeparatedFormatReader(ReadBuffer & in_, const FormatSettings & format_settings, bool is_raw_);
bool readField(IColumn & column, const DataTypePtr & type,
const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) override;
@ -36,18 +44,34 @@ private:
void skipFieldDelimiter() override;
void skipRowEndDelimiter() override;
std::vector<String> readHeaderRow();
std::vector<String> readNames() override { return readHeaderRow(); }
std::vector<String> readTypes() override { return readHeaderRow(); }
std::vector<String> readRow();
std::vector<String> readNames() override { return readRow(); }
std::vector<String> readTypes() override { return readRow(); }
String readFieldIntoString();
void checkNullValueForNonNullable(DataTypePtr type) override;
bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer & out) override;
bool parseRowEndWithDiagnosticInfo(WriteBuffer & out) override;
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
FormatSettings::EscapingRule getEscapingRule()
{
return is_raw ? FormatSettings::EscapingRule::Raw : FormatSettings::EscapingRule::Escaped;
}
private:
bool is_raw;
bool first_row = true;
};
class TabSeparatedSchemaReader : public FormatWithNamesAndTypesSchemaReader
{
public:
TabSeparatedSchemaReader(ReadBuffer & in_, bool with_names_, bool with_types_, bool is_raw_, const FormatSettings & format_settings);
private:
DataTypes readRowAndGetDataTypes() override;
TabSeparatedFormatReader reader;
};
}

View File

@ -4,7 +4,6 @@
#include <Formats/EscapingRuleUtils.h>
#include <IO/Operators.h>
#include <DataTypes/DataTypeNothing.h>
#include <Interpreters/Context.h>
#include <DataTypes/Serializations/SerializationNullable.h>
namespace DB
@ -19,6 +18,12 @@ extern const int CANNOT_PARSE_QUOTED_STRING;
extern const int SYNTAX_ERROR;
}
[[noreturn]] static void throwUnexpectedEof(size_t row_num)
{
throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
ErrorCodes::CANNOT_READ_ALL_DATA);
}
TemplateRowInputFormat::TemplateRowInputFormat(
const Block & header_,
@ -41,37 +46,13 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::uniqu
: RowInputFormatWithDiagnosticInfo(header_, *buf_, params_), buf(std::move(buf_)), data_types(header_.getDataTypes()),
settings(std::move(settings_)), ignore_spaces(ignore_spaces_),
format(std::move(format_)), row_format(std::move(row_format_)),
default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(std::move(row_between_delimiter_))
default_csv_delimiter(settings.csv.delimiter), row_between_delimiter(row_between_delimiter_),
format_reader(std::make_unique<TemplateFormatReader>(*buf, ignore_spaces_, format, row_format, row_between_delimiter, settings))
{
/// Validate format string for result set
bool has_data = false;
for (size_t i = 0; i < format.columnsCount(); ++i)
{
if (format.format_idx_to_column_idx[i])
{
if (*format.format_idx_to_column_idx[i] != 0)
format.throwInvalidFormat("Invalid input part", i);
if (has_data)
format.throwInvalidFormat("${data} can occur only once", i);
if (format.escaping_rules[i] != EscapingRule::None)
format.throwInvalidFormat("${data} must have empty or None deserialization type", i);
has_data = true;
format_data_idx = i;
}
else
{
if (format.escaping_rules[i] == EscapingRule::XML)
format.throwInvalidFormat("XML deserialization is not supported", i);
}
}
/// Validate format string for rows
std::vector<UInt8> column_in_format(header_.columns(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.escaping_rules[i] == EscapingRule::XML)
row_format.throwInvalidFormat("XML deserialization is not supported", i);
if (row_format.format_idx_to_column_idx[i])
{
if (header_.columns() <= *row_format.format_idx_to_column_idx[i])
@ -94,69 +75,7 @@ TemplateRowInputFormat::TemplateRowInputFormat(const Block & header_, std::uniqu
void TemplateRowInputFormat::readPrefix()
{
size_t last_successfully_parsed_idx = 0;
try
{
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format_data_idx);
}
catch (Exception & e)
{
format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx);
}
}
/// Asserts delimiters and skips fields in prefix or suffix.
/// tryReadPrefixOrSuffix<bool>(...) is used in checkForSuffix() to avoid throwing an exception after read of each row
/// (most likely false will be returned on first call of checkString(...))
template <typename ReturnType>
ReturnType TemplateRowInputFormat::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
while (input_part_beg < input_part_end)
{
skipSpaces();
if constexpr (throw_exception)
skipField(format.escaping_rules[input_part_beg]);
else
{
try
{
skipField(format.escaping_rules[input_part_beg]);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
/// If it's parsing error, then suffix is not found
return ReturnType(false);
}
}
++input_part_beg;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
}
if constexpr (!throw_exception)
return ReturnType(true);
format_reader->readPrefix();
}
bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension & extra)
@ -165,9 +84,7 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
if (unlikely(end_of_stream))
return false;
skipSpaces();
if (unlikely(checkForSuffix()))
if (unlikely(format_reader->checkForSuffix()))
{
end_of_stream = true;
return false;
@ -176,27 +93,24 @@ bool TemplateRowInputFormat::readRow(MutableColumns & columns, RowReadExtension
updateDiagnosticInfo();
if (likely(row_num != 1))
assertString(row_between_delimiter, *buf);
format_reader->skipRowBetweenDelimiter();
extra.read_columns.assign(columns.size(), false);
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
skipSpaces();
assertString(row_format.delimiters[i], *buf);
skipSpaces();
format_reader->skipDelimiter(i);
if (row_format.format_idx_to_column_idx[i])
{
size_t col_idx = *row_format.format_idx_to_column_idx[i];
extra.read_columns[col_idx] = deserializeField(data_types[col_idx], serializations[col_idx], *columns[col_idx], i);
}
else
skipField(row_format.escaping_rules[i]);
format_reader->skipField(row_format.escaping_rules[i]);
}
skipSpaces();
assertString(row_format.delimiters.back(), *buf);
format_reader->skipRowEndDelimiter();
for (const auto & idx : always_default_columns)
data_types[idx]->insertDefaultInto(*columns[idx]);
@ -219,65 +133,21 @@ bool TemplateRowInputFormat::deserializeField(const DataTypePtr & type,
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throwUnexpectedEof(row_num);
throw;
}
}
void TemplateRowInputFormat::skipField(TemplateRowInputFormat::EscapingRule escaping_rule)
{
try
{
skipFieldByEscapingRule(*buf, escaping_rule, settings);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof();
throw;
}
}
/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF.
/// Otherwise returns false
bool TemplateRowInputFormat::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{*buf};
bool suffix_found = false;
size_t last_successfully_parsed_idx = format_data_idx + 1;
try
{
suffix_found = tryReadPrefixOrSuffix<bool>(last_successfully_parsed_idx, format.columnsCount());
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
}
if (unlikely(suffix_found))
{
skipSpaces();
if (buf->eof())
return true;
}
buf->rollbackToCheckpoint();
return false;
}
bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out)
{
out << "Suffix does not match: ";
size_t last_successfully_parsed_idx = format_data_idx + 1;
size_t last_successfully_parsed_idx = format_reader->getFormatDataIdx() + 1;
const ReadBuffer::Position row_begin_pos = buf->position();
bool caught = false;
try
{
PeekableReadBufferCheckpoint checkpoint{*buf, true};
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format.columnsCount());
format_reader->tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format.columnsCount());
}
catch (Exception & e)
{
@ -309,7 +179,7 @@ bool TemplateRowInputFormat::parseRowAndPrintDiagnosticInfo(MutableColumns & col
if (!parseDelimiterWithDiagnosticInfo(out, *buf, row_format.delimiters[i], "delimiter before field " + std::to_string(i), ignore_spaces))
return false;
skipSpaces();
format_reader->skipSpaces();
if (row_format.format_idx_to_column_idx[i])
{
const auto & header = getPort().getHeader();
@ -364,7 +234,7 @@ void TemplateRowInputFormat::tryDeserializeField(const DataTypePtr & type, IColu
if (index)
deserializeField(type, serializations[*index], column, file_column);
else
skipField(row_format.escaping_rules[file_column]);
format_reader->skipField(row_format.escaping_rules[file_column]);
}
bool TemplateRowInputFormat::isGarbageAfterField(size_t, ReadBuffer::Position)
@ -387,13 +257,6 @@ void TemplateRowInputFormat::syncAfterError()
/// It will cause another parsing error.
}
void TemplateRowInputFormat::throwUnexpectedEof()
{
throw ParsingException("Unexpected EOF while parsing row " + std::to_string(row_num) + ". "
"Maybe last row has wrong format or input doesn't contain specified suffix before EOF.",
ErrorCodes::CANNOT_READ_ALL_DATA);
}
void TemplateRowInputFormat::resetParser()
{
RowInputFormatWithDiagnosticInfo::resetParser();
@ -407,15 +270,233 @@ void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_)
IInputFormat::setReadBuffer(*buf);
}
void registerInputFormatTemplate(FormatFactory & factory)
TemplateFormatReader::TemplateFormatReader(
PeekableReadBuffer & buf_,
bool ignore_spaces_,
const ParsedTemplateFormatString & format_,
const ParsedTemplateFormatString & row_format_,
std::string row_between_delimiter_,
const FormatSettings & format_settings_)
: buf(&buf_)
, ignore_spaces(ignore_spaces_)
, format(format_)
, row_format(row_format_)
, row_between_delimiter(row_between_delimiter_)
, format_settings(format_settings_)
{
for (bool ignore_spaces : {false, true})
/// Validate format string for result set
bool has_data = false;
for (size_t i = 0; i < format.columnsCount(); ++i)
{
factory.registerInputFormat(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
if (format.format_idx_to_column_idx[i])
{
if (*format.format_idx_to_column_idx[i] != 0)
format.throwInvalidFormat("Invalid input part", i);
if (has_data)
format.throwInvalidFormat("${data} can occur only once", i);
if (format.escaping_rules[i] != EscapingRule::None)
format.throwInvalidFormat("${data} must have empty or None deserialization type", i);
has_data = true;
format_data_idx = i;
}
else
{
if (format.escaping_rules[i] == EscapingRule::XML)
format.throwInvalidFormat("XML deserialization is not supported", i);
}
}
/// Validate format string for rows
for (size_t i = 0; i < row_format.columnsCount(); ++i)
{
if (row_format.escaping_rules[i] == EscapingRule::XML)
row_format.throwInvalidFormat("XML deserialization is not supported", i);
}
}
void TemplateFormatReader::readPrefix()
{
size_t last_successfully_parsed_idx = 0;
try
{
tryReadPrefixOrSuffix<void>(last_successfully_parsed_idx, format_data_idx);
}
catch (Exception & e)
{
format.throwInvalidFormat(e.message() + " While parsing prefix", last_successfully_parsed_idx);
}
}
void TemplateFormatReader::skipField(EscapingRule escaping_rule)
{
try
{
skipFieldByEscapingRule(*buf, escaping_rule, format_settings);
}
catch (Exception & e)
{
if (e.code() == ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF)
throwUnexpectedEof(row_num);
throw;
}
}
/// Asserts delimiters and skips fields in prefix or suffix.
/// tryReadPrefixOrSuffix<bool>(...) is used in checkForSuffix() to avoid throwing an exception after read of each row
/// (most likely false will be returned on first call of checkString(...))
template <typename ReturnType>
ReturnType TemplateFormatReader::tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end)
{
static constexpr bool throw_exception = std::is_same_v<ReturnType, void>;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
while (input_part_beg < input_part_end)
{
skipSpaces();
if constexpr (throw_exception)
skipField(format.escaping_rules[input_part_beg]);
else
{
try
{
skipField(format.escaping_rules[input_part_beg]);
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
/// If it's parsing error, then suffix is not found
return ReturnType(false);
}
}
++input_part_beg;
skipSpaces();
if constexpr (throw_exception)
assertString(format.delimiters[input_part_beg], *buf);
else
{
if (likely(!checkString(format.delimiters[input_part_beg], *buf)))
return ReturnType(false);
}
}
if constexpr (!throw_exception)
return ReturnType(true);
}
/// Returns true if all rows have been read i.e. there are only suffix and spaces (if ignore_spaces == true) before EOF.
/// Otherwise returns false
bool TemplateFormatReader::checkForSuffix()
{
PeekableReadBufferCheckpoint checkpoint{*buf};
bool suffix_found = false;
size_t last_successfully_parsed_idx = format_data_idx + 1;
try
{
suffix_found = tryReadPrefixOrSuffix<bool>(last_successfully_parsed_idx, format.columnsCount());
}
catch (const Exception & e)
{
if (e.code() != ErrorCodes::ATTEMPT_TO_READ_AFTER_EOF &&
e.code() != ErrorCodes::CANNOT_PARSE_ESCAPE_SEQUENCE &&
e.code() != ErrorCodes::CANNOT_PARSE_QUOTED_STRING)
throw;
}
if (unlikely(suffix_found))
{
skipSpaces();
if (buf->eof())
return true;
}
buf->rollbackToCheckpoint();
return false;
}
void TemplateFormatReader::skipDelimiter(size_t index)
{
skipSpaces();
assertString(row_format.delimiters[index], *buf);
skipSpaces();
}
void TemplateFormatReader::skipRowEndDelimiter()
{
++row_num;
skipSpaces();
assertString(row_format.delimiters.back(), *buf);
skipSpaces();
}
void TemplateFormatReader::skipRowBetweenDelimiter()
{
skipSpaces();
assertString(row_between_delimiter, *buf);
skipSpaces();
}
TemplateSchemaReader::TemplateSchemaReader(
ReadBuffer & in_,
bool ignore_spaces_,
const ParsedTemplateFormatString & format_,
const ParsedTemplateFormatString & row_format_,
std::string row_between_delimiter,
const FormatSettings & format_settings_,
ContextPtr context_)
: IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference)
, buf(in_)
, format(format_)
, row_format(row_format_)
, format_settings(format_settings_)
, context(context_)
, format_reader(buf, ignore_spaces_, format, row_format, row_between_delimiter, format_settings)
{
setColumnNames(row_format.column_names);
}
DataTypes TemplateSchemaReader::readRowAndGetDataTypes()
{
if (first_row)
format_reader.readPrefix();
if (format_reader.checkForSuffix())
return {};
if (first_row)
first_row = false;
else
format_reader.skipRowBetweenDelimiter();
DataTypes data_types;
data_types.reserve(row_format.columnsCount());
String field;
for (size_t i = 0; i != row_format.columnsCount(); ++i)
{
format_reader.skipDelimiter(i);
if (row_format.escaping_rules[i] == FormatSettings::EscapingRule::CSV)
format_settings.csv.delimiter = row_format.delimiters[i + 1].empty() ? format_settings.csv.delimiter : row_format.delimiters[i + 1].front();
field = readFieldByEscapingRule(buf, row_format.escaping_rules[i], format_settings);
data_types.push_back(determineDataTypeByEscapingRule(field, format_settings, row_format.escaping_rules[i], context));
}
format_reader.skipRowEndDelimiter();
return data_types;
}
static ParsedTemplateFormatString fillResultSetFormat(const FormatSettings & settings)
{
ParsedTemplateFormatString resultset_format;
if (settings.template_settings.resultset_format.empty())
@ -440,16 +521,55 @@ void registerInputFormatTemplate(FormatFactory & factory)
ErrorCodes::SYNTAX_ERROR);
});
}
return resultset_format;
}
ParsedTemplateFormatString row_format = ParsedTemplateFormatString(
FormatSchemaInfo(settings.template_settings.row_format, "Template", false,
settings.schema.is_server, settings.schema.format_schema_path),
[&](const String & colName) -> std::optional<size_t>
static ParsedTemplateFormatString fillRowFormat(const FormatSettings & settings, ParsedTemplateFormatString::ColumnIdxGetter idx_getter, bool allow_indexes)
{
return ParsedTemplateFormatString(
FormatSchemaInfo(
settings.template_settings.row_format, "Template", false, settings.schema.is_server, settings.schema.format_schema_path),
idx_getter, allow_indexes);
}
void registerInputFormatTemplate(FormatFactory & factory)
{
for (bool ignore_spaces : {false, true})
{
factory.registerInputFormat(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [=](
ReadBuffer & buf,
const Block & sample,
IRowInputFormat::Params params,
const FormatSettings & settings)
{
auto idx_getter = [&](const String & colName) -> std::optional<size_t>
{
return sample.getPositionByName(colName);
});
};
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, settings.template_settings.row_between_delimiter);
return std::make_shared<TemplateRowInputFormat>(
sample,
buf,
params,
settings,
ignore_spaces,
fillResultSetFormat(settings),
fillRowFormat(settings, idx_getter, true),
settings.template_settings.row_between_delimiter);
});
}
}
void registerTemplateSchemaReader(FormatFactory & factory)
{
for (bool ignore_spaces : {false, true})
{
factory.registerSchemaReader(ignore_spaces ? "TemplateIgnoreSpaces" : "Template", [ignore_spaces](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context)
{
size_t index = 0;
auto idx_getter = [&](const String &) -> std::optional<size_t> { return index++; };
auto row_format = fillRowFormat(settings, idx_getter, false);
return std::make_shared<TemplateSchemaReader>(buf, ignore_spaces, fillResultSetFormat(settings), row_format, settings.template_settings.row_between_delimiter, settings, context);
});
}
}

View File

@ -2,15 +2,19 @@
#include <Core/Block.h>
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/ParsedTemplateFormatString.h>
#include <IO/ReadHelpers.h>
#include <IO/PeekableReadBuffer.h>
#include <Interpreters/Context.h>
namespace DB
{
class TemplateFormatReader;
class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
{
using EscapingRule = FormatSettings::EscapingRule;
@ -40,14 +44,6 @@ private:
bool deserializeField(const DataTypePtr & type,
const SerializationPtr & serialization, IColumn & column, size_t file_column);
void skipField(EscapingRule escaping_rule);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
template <typename ReturnType = void>
ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end);
bool checkForSuffix();
[[noreturn]] void throwUnexpectedEof();
bool parseRowAndPrintDiagnosticInfo(MutableColumns & columns, WriteBuffer & out) override;
void tryDeserializeField(const DataTypePtr & type, IColumn & column, size_t file_column) override;
@ -63,12 +59,76 @@ private:
const ParsedTemplateFormatString format;
const ParsedTemplateFormatString row_format;
size_t format_data_idx;
bool end_of_stream = false;
std::vector<size_t> always_default_columns;
const char default_csv_delimiter;
const std::string row_between_delimiter;
std::unique_ptr<TemplateFormatReader> format_reader;
};
class TemplateFormatReader
{
using EscapingRule = FormatSettings::EscapingRule;
public:
TemplateFormatReader(
PeekableReadBuffer & buf_,
bool ignore_spaces_,
const ParsedTemplateFormatString & format_,
const ParsedTemplateFormatString & row_format_,
std::string row_between_delimiter,
const FormatSettings & format_settings_);
void readPrefix();
void skipField(EscapingRule escaping_rule);
inline void skipSpaces() { if (ignore_spaces) skipWhitespaceIfAny(*buf); }
template <typename ReturnType = void>
ReturnType tryReadPrefixOrSuffix(size_t & input_part_beg, size_t input_part_end);
bool checkForSuffix();
void setReadBuffer(PeekableReadBuffer & buf_) { buf = &buf_; }
void skipDelimiter(size_t index);
void skipRowEndDelimiter();
void skipRowBetweenDelimiter();
size_t getFormatDataIdx() const { return format_data_idx; }
private:
PeekableReadBuffer * buf;
bool ignore_spaces;
const ParsedTemplateFormatString & format;
const ParsedTemplateFormatString & row_format;
const std::string row_between_delimiter;
const FormatSettings & format_settings;
size_t format_data_idx;
size_t row_num;
};
class TemplateSchemaReader : public IRowSchemaReader
{
public:
TemplateSchemaReader(ReadBuffer & in_,
bool ignore_spaces_,
const ParsedTemplateFormatString & format_,
const ParsedTemplateFormatString & row_format_,
std::string row_between_delimiter,
const FormatSettings & format_settings_,
ContextPtr context_);
DataTypes readRowAndGetDataTypes() override;
private:
PeekableReadBuffer buf;
const ParsedTemplateFormatString format;
const ParsedTemplateFormatString row_format;
FormatSettings format_settings;
ContextPtr context;
TemplateFormatReader format_reader;
bool first_row = true;
};
bool parseDelimiterWithDiagnosticInfo(WriteBuffer & out, ReadBuffer & buf, const String & delimiter, const String & description, bool skip_spaces);

View File

@ -5,6 +5,7 @@
#include <Parsers/TokenIterator.h>
#include <Processors/Formats/Impl/ValuesBlockInputFormat.h>
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Core/Block.h>
#include <base/find_symbols.h>
#include <Common/typeid_cast.h>
@ -15,6 +16,7 @@
#include <DataTypes/DataTypeArray.h>
#include <DataTypes/DataTypeMap.h>
#include <base/logger_useful.h>
namespace DB
{
@ -286,6 +288,50 @@ namespace
}
}
/// Can be used in fileSegmentationEngine for parallel parsing of Values
static bool skipToNextRow(PeekableReadBuffer * buf, size_t min_chunk_bytes, int balance)
{
skipWhitespaceIfAny(*buf);
if (buf->eof() || *buf->position() == ';')
return false;
bool quoted = false;
size_t chunk_begin_buf_count = buf->count();
while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes))
{
buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end());
if (buf->position() == buf->buffer().end())
continue;
if (*buf->position() == '\\')
{
++buf->position();
if (!buf->eof())
++buf->position();
}
else if (*buf->position() == '\'')
{
quoted ^= true;
++buf->position();
}
else if (*buf->position() == ')')
{
++buf->position();
if (!quoted)
--balance;
}
else if (*buf->position() == '(')
{
++buf->position();
if (!quoted)
++balance;
}
}
if (!buf->eof() && *buf->position() == ',')
++buf->position();
return true;
}
bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx)
{
const Block & header = getPort().getHeader();
@ -293,7 +339,7 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
auto settings = context->getSettingsRef();
/// We need continuous memory containing the expression to use Lexer
skipToNextRow(0, 1);
skipToNextRow(buf.get(), 0, 1);
buf->makeContinuousMemoryFromCheckpointToPos();
buf->rollbackToCheckpoint();
@ -437,50 +483,6 @@ bool ValuesBlockInputFormat::parseExpression(IColumn & column, size_t column_idx
return true;
}
/// Can be used in fileSegmentationEngine for parallel parsing of Values
bool ValuesBlockInputFormat::skipToNextRow(size_t min_chunk_bytes, int balance)
{
skipWhitespaceIfAny(*buf);
if (buf->eof() || *buf->position() == ';')
return false;
bool quoted = false;
size_t chunk_begin_buf_count = buf->count();
while (!buf->eof() && (balance || buf->count() - chunk_begin_buf_count < min_chunk_bytes))
{
buf->position() = find_first_symbols<'\\', '\'', ')', '('>(buf->position(), buf->buffer().end());
if (buf->position() == buf->buffer().end())
continue;
if (*buf->position() == '\\')
{
++buf->position();
if (!buf->eof())
++buf->position();
}
else if (*buf->position() == '\'')
{
quoted ^= true;
++buf->position();
}
else if (*buf->position() == ')')
{
++buf->position();
if (!quoted)
--balance;
}
else if (*buf->position() == '(')
{
++buf->position();
if (!quoted)
++balance;
}
}
if (!buf->eof() && *buf->position() == ',')
++buf->position();
return true;
}
void ValuesBlockInputFormat::assertDelimiterAfterValue(size_t column_idx)
{
if (unlikely(!checkDelimiterAfterValue(column_idx)))
@ -559,6 +561,63 @@ void ValuesBlockInputFormat::setReadBuffer(ReadBuffer & in_)
IInputFormat::setReadBuffer(*buf);
}
ValuesSchemaReader::ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, ContextPtr context_)
: IRowSchemaReader(buf, format_settings_.max_rows_to_read_for_schema_inference), buf(in_), context(context_)
{
}
DataTypes ValuesSchemaReader::readRowAndGetDataTypes()
{
if (first_row)
{
skipBOMIfExists(buf);
first_row = false;
}
skipWhitespaceIfAny(buf);
if (buf.eof())
return {};
assertChar('(', buf);
PeekableReadBufferCheckpoint checkpoint(buf);
skipToNextRow(&buf, 0, 1);
buf.makeContinuousMemoryFromCheckpointToPos();
buf.rollbackToCheckpoint();
Tokens tokens(buf.position(), buf.buffer().end());
IParser::Pos token_iterator(tokens, context->getSettingsRef().max_parser_depth);
DataTypes data_types;
bool finish = false;
while (!finish)
{
Expected expected;
ASTPtr ast;
bool parsed = parser.parse(token_iterator, ast, expected);
/// Consider delimiter after value (',' or ')') as part of expression
parsed &= token_iterator->type == TokenType::Comma || token_iterator->type == TokenType::ClosingRoundBracket;
if (!parsed)
throw Exception(ErrorCodes::SYNTAX_ERROR, "Cannot parse expression here: {}, token: {}",
String(buf.position(), std::min(SHOW_CHARS_ON_SYNTAX_ERROR, buf.buffer().end() - buf.position())), String(token_iterator.get().begin, token_iterator.get().end));
std::pair<Field, DataTypePtr> result = evaluateConstantExpression(ast, context);
data_types.push_back(generalizeDataType(result.second));
if (token_iterator->type == TokenType::ClosingRoundBracket)
finish = true;
++token_iterator;
buf.position() = const_cast<char *>(token_iterator->begin);
}
skipWhitespaceIfAny(buf);
if (!buf.eof() && *buf.position() == ',')
++buf.position();
return data_types;
}
void registerInputFormatValues(FormatFactory & factory)
{
factory.registerInputFormat("Values", [](
@ -571,4 +630,12 @@ void registerInputFormatValues(FormatFactory & factory)
});
}
void registerValuesSchemaReader(FormatFactory & factory)
{
factory.registerSchemaReader("Values", [](ReadBuffer & buf, const FormatSettings & settings, ContextPtr context)
{
return std::make_shared<ValuesSchemaReader>(buf, settings, context);
});
}
}

View File

@ -7,6 +7,7 @@
#include <Parsers/ExpressionListParsers.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/IRowInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Processors/Formats/Impl/ConstantExpressionTemplate.h>
namespace DB
@ -68,8 +69,6 @@ private:
void readPrefix();
void readSuffix();
bool skipToNextRow(size_t min_chunk_bytes = 0, int balance = 0);
std::unique_ptr<PeekableReadBuffer> buf;
const RowInputFormatParams params;
@ -95,4 +94,18 @@ private:
BlockMissingValues block_missing_values;
};
class ValuesSchemaReader : public IRowSchemaReader
{
public:
ValuesSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings, ContextPtr context_);
private:
DataTypes readRowAndGetDataTypes() override;
PeekableReadBuffer buf;
ContextPtr context;
ParserExpression parser;
bool first_row = true;
};
}

View File

@ -1,5 +1,7 @@
#include <Processors/Formats/RowInputFormatWithNamesAndTypes.h>
#include <Processors/Formats/ISchemaReader.h>
#include <DataTypes/DataTypeNothing.h>
#include <DataTypes/DataTypeFactory.h>
#include <IO/ReadHelpers.h>
#include <IO/Operators.h>
@ -9,6 +11,7 @@ namespace DB
namespace ErrorCodes
{
extern const int INCORRECT_DATA;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes(
@ -17,8 +20,13 @@ RowInputFormatWithNamesAndTypes::RowInputFormatWithNamesAndTypes(
const Params & params_,
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_), format_settings(format_settings_), with_names(with_names_), with_types(with_types_)
const FormatSettings & format_settings_,
std::unique_ptr<FormatWithNamesAndTypesReader> format_reader_)
: RowInputFormatWithDiagnosticInfo(header_, in_, params_)
, format_settings(format_settings_)
, with_names(with_names_)
, with_types(with_types_)
, format_reader(std::move(format_reader_))
{
const auto & sample = getPort().getHeader();
size_t num_columns = sample.columns();
@ -88,7 +96,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix()
}
/// Skip prefix before names and types.
skipPrefixBeforeHeader();
format_reader->skipPrefixBeforeHeader();
/// This is a bit of abstraction leakage, but we need it in parallel parsing:
/// we check if this InputFormat is working with the "real" beginning of the data.
@ -97,7 +105,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix()
if (format_settings.with_names_use_header)
{
std::vector<bool> read_columns(data_types.size(), false);
auto column_names = readNames();
auto column_names = format_reader->readNames();
for (const auto & name : column_names)
addInputColumn(name, read_columns);
@ -110,7 +118,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix()
else
{
setupAllColumnsByTableSchema();
skipNames();
format_reader->skipNames();
}
}
else if (!column_mapping->is_set)
@ -119,10 +127,10 @@ void RowInputFormatWithNamesAndTypes::readPrefix()
if (with_types)
{
/// Skip delimiter between names and types.
skipRowBetweenDelimiter();
format_reader->skipRowBetweenDelimiter();
if (format_settings.with_types_use_header)
{
auto types = readTypes();
auto types = format_reader->readTypes();
if (types.size() != column_mapping->column_indexes_for_input_fields.size())
throw Exception(
ErrorCodes::INCORRECT_DATA,
@ -143,7 +151,7 @@ void RowInputFormatWithNamesAndTypes::readPrefix()
}
}
else
skipTypes();
format_reader->skipTypes();
}
}
@ -161,7 +169,7 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE
if (unlikely(end_of_stream))
return false;
if (unlikely(checkForSuffix()))
if (unlikely(format_reader->checkForSuffix()))
{
end_of_stream = true;
return false;
@ -170,9 +178,9 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE
updateDiagnosticInfo();
if (likely(row_num != 1 || (getCurrentUnitNumber() == 0 && (with_names || with_types))))
skipRowBetweenDelimiter();
format_reader->skipRowBetweenDelimiter();
skipRowStartDelimiter();
format_reader->skipRowStartDelimiter();
ext.read_columns.resize(data_types.size());
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
@ -180,20 +188,20 @@ bool RowInputFormatWithNamesAndTypes::readRow(MutableColumns & columns, RowReadE
const auto & column_index = column_mapping->column_indexes_for_input_fields[file_column];
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
if (column_index)
ext.read_columns[*column_index] = readField(
ext.read_columns[*column_index] = format_reader->readField(
*columns[*column_index],
data_types[*column_index],
serializations[*column_index],
is_last_file_column,
column_mapping->names_of_columns[file_column]);
else
skipField(file_column);
format_reader->skipField(file_column);
if (!is_last_file_column)
skipFieldDelimiter();
format_reader->skipFieldDelimiter();
}
skipRowEndDelimiter();
format_reader->skipRowEndDelimiter();
insertDefaultsForNotSeenColumns(columns, ext);
@ -218,13 +226,13 @@ void RowInputFormatWithNamesAndTypes::tryDeserializeField(const DataTypePtr & ty
const auto & index = column_mapping->column_indexes_for_input_fields[file_column];
if (index)
{
checkNullValueForNonNullable(type);
format_reader->checkNullValueForNonNullable(type);
const bool is_last_file_column = file_column + 1 == column_mapping->column_indexes_for_input_fields.size();
readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]);
format_reader->readField(column, type, serializations[*index], is_last_file_column, column_mapping->names_of_columns[file_column]);
}
else
{
skipField(file_column);
format_reader->skipField(file_column);
}
}
@ -236,13 +244,13 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu
return false;
}
if (!tryParseSuffixWithDiagnosticInfo(out))
if (!format_reader->tryParseSuffixWithDiagnosticInfo(out))
return false;
if (likely(row_num != 1) && !parseRowBetweenDelimiterWithDiagnosticInfo(out))
if (likely(row_num != 1) && !format_reader->parseRowBetweenDelimiterWithDiagnosticInfo(out))
return false;
if (!parseRowStartWithDiagnosticInfo(out))
if (!format_reader->parseRowStartWithDiagnosticInfo(out))
return false;
for (size_t file_column = 0; file_column < column_mapping->column_indexes_for_input_fields.size(); ++file_column)
@ -266,22 +274,68 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu
/// Delimiters
if (file_column + 1 != column_mapping->column_indexes_for_input_fields.size())
{
if (!parseFieldDelimiterWithDiagnosticInfo(out))
if (!format_reader->parseFieldDelimiterWithDiagnosticInfo(out))
return false;
}
}
return parseRowEndWithDiagnosticInfo(out);
return format_reader->parseRowEndWithDiagnosticInfo(out);
}
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine)
bool RowInputFormatWithNamesAndTypes::isGarbageAfterField(size_t index, ReadBuffer::Position pos)
{
factory.registerFileSegmentationEngine(base_format_name, segmentation_engine);
factory.registerFileSegmentationEngine(base_format_name + "WithNames", segmentation_engine);
factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", segmentation_engine);
return format_reader->isGarbageAfterField(index, pos);
}
void RowInputFormatWithNamesAndTypes::setReadBuffer(ReadBuffer & in_)
{
format_reader->setReadBuffer(in_);
IInputFormat::setReadBuffer(in_);
}
FormatWithNamesAndTypesSchemaReader::FormatWithNamesAndTypesSchemaReader(
ReadBuffer & in_,
size_t max_rows_to_read_,
bool with_names_,
bool with_types_,
FormatWithNamesAndTypesReader * format_reader_,
DataTypePtr default_type_)
: IRowSchemaReader(in_, max_rows_to_read_, default_type_), with_names(with_names_), with_types(with_types_), format_reader(format_reader_)
{
}
NamesAndTypesList FormatWithNamesAndTypesSchemaReader::readSchema()
{
if (with_names || with_types)
skipBOMIfExists(in);
format_reader->skipPrefixBeforeHeader();
Names names;
if (with_names)
names = format_reader->readNames();
if (with_types)
{
format_reader->skipRowBetweenDelimiter();
std::vector<String> data_type_names = format_reader->readTypes();
if (data_type_names.size() != names.size())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The number of column names {} differs with the number of types {}", names.size(), data_type_names.size());
NamesAndTypesList result;
for (size_t i = 0; i != data_type_names.size(); ++i)
result.emplace_back(names[i], DataTypeFactory::instance().get(data_type_names[i]));
return result;
}
if (!names.empty())
setColumnNames(names);
/// We should determine types by reading rows with data. Use the implementation from IRowSchemaReader.
return IRowSchemaReader::readSchema();
}
}

View File

@ -1,12 +1,15 @@
#pragma once
#include <Processors/Formats/RowInputFormatWithDiagnosticInfo.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Formats/FormatSettings.h>
#include <Formats/FormatFactory.h>
namespace DB
{
class FormatWithNamesAndTypesReader;
/// Base class for input formats with -WithNames and -WithNamesAndTypes suffixes.
/// It accepts 2 parameters in constructor - with_names and with_types and implements
/// input format depending on them:
@ -20,7 +23,7 @@ namespace DB
/// then reads/skips types. So you can this invariant.
class RowInputFormatWithNamesAndTypes : public RowInputFormatWithDiagnosticInfo
{
public:
protected:
/** with_names - in the first line the header with column names
* with_types - in the second line the header with column names
*/
@ -28,44 +31,14 @@ public:
const Block & header_,
ReadBuffer & in_,
const Params & params_,
bool with_names_, bool with_types_, const FormatSettings & format_settings_);
bool with_names_,
bool with_types_,
const FormatSettings & format_settings_,
std::unique_ptr<FormatWithNamesAndTypesReader> format_reader_);
void resetParser() override;
protected:
/// Read single field from input. Return false if there was no real value and we inserted default value.
virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0;
/// Skip single field, it's used to skip unknown columns.
virtual void skipField(size_t file_column) = 0;
/// Skip the whole row with names.
virtual void skipNames() = 0;
/// Skip the whole row with types.
virtual void skipTypes() = 0;
/// Skip delimiters, if any.
virtual void skipPrefixBeforeHeader() {}
virtual void skipRowStartDelimiter() {}
virtual void skipFieldDelimiter() {}
virtual void skipRowEndDelimiter() {}
virtual void skipRowBetweenDelimiter() {}
/// Check suffix.
virtual bool checkForSuffix() { return in->eof(); }
/// Methods for parsing with diagnostic info.
virtual void checkNullValueForNonNullable(DataTypePtr) {}
virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; }
virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; }
virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;}
virtual bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer &) { return true;}
virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; }
bool isGarbageAfterField(size_t, ReadBuffer::Position) override {return false; }
/// Read row with names and return the list of them.
virtual std::vector<String> readNames() = 0;
/// Read row with types and return the list of them.
virtual std::vector<String> readTypes() = 0;
bool isGarbageAfterField(size_t index, ReadBuffer::Position pos) override;
void setReadBuffer(ReadBuffer & in_) override;
const FormatSettings format_settings;
DataTypes data_types;
@ -84,10 +57,90 @@ private:
bool with_names;
bool with_types;
std::unique_ptr<FormatWithNamesAndTypesReader> format_reader;
std::unordered_map<String, size_t> column_indexes_by_names;
};
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine);
/// Base class for parsing data in input formats with -WithNames and -WithNamesAndTypes suffixes.
/// Used for reading/skipping names/types/delimiters in specific format.
class FormatWithNamesAndTypesReader
{
public:
explicit FormatWithNamesAndTypesReader(ReadBuffer & in_, const FormatSettings & format_settings_) : in(&in_), format_settings(format_settings_) {}
/// Read single field from input. Return false if there was no real value and we inserted default value.
virtual bool readField(IColumn & column, const DataTypePtr & type, const SerializationPtr & serialization, bool is_last_file_column, const String & column_name) = 0;
/// Methods for parsing with diagnostic info.
virtual void checkNullValueForNonNullable(DataTypePtr) {}
virtual bool parseRowStartWithDiagnosticInfo(WriteBuffer &) { return true; }
virtual bool parseFieldDelimiterWithDiagnosticInfo(WriteBuffer &) { return true; }
virtual bool parseRowEndWithDiagnosticInfo(WriteBuffer &) { return true;}
virtual bool parseRowBetweenDelimiterWithDiagnosticInfo(WriteBuffer &) { return true;}
virtual bool tryParseSuffixWithDiagnosticInfo(WriteBuffer &) { return true; }
virtual bool isGarbageAfterField(size_t, ReadBuffer::Position) { return false; }
/// Read row with names and return the list of them.
virtual std::vector<String> readNames() = 0;
/// Read row with types and return the list of them.
virtual std::vector<String> readTypes() = 0;
/// Skip single field, it's used to skip unknown columns.
virtual void skipField(size_t file_column) = 0;
/// Skip the whole row with names.
virtual void skipNames() = 0;
/// Skip the whole row with types.
virtual void skipTypes() = 0;
/// Skip delimiters, if any.
virtual void skipPrefixBeforeHeader() {}
virtual void skipRowStartDelimiter() {}
virtual void skipFieldDelimiter() {}
virtual void skipRowEndDelimiter() {}
virtual void skipRowBetweenDelimiter() {}
/// Check suffix.
virtual bool checkForSuffix() { return in->eof(); }
const FormatSettings & getFormatSettings() const { return format_settings; }
virtual void setReadBuffer(ReadBuffer & in_) { in = &in_; }
virtual ~FormatWithNamesAndTypesReader() = default;
protected:
ReadBuffer * in;
const FormatSettings format_settings;
};
/// Base class for schema inference for formats with -WithNames and -WithNamesAndTypes suffixes.
/// For formats with -WithNamesAndTypes suffix the schema will be determined by first two rows.
/// For formats with -WithNames suffix the names of columns will be determined by the first row
/// and types of columns by the rows with data.
/// For formats without suffixes default column names will be used
/// and types will be determined by the rows with data.
class FormatWithNamesAndTypesSchemaReader : public IRowSchemaReader
{
public:
FormatWithNamesAndTypesSchemaReader(
ReadBuffer & in,
size_t max_rows_to_read_,
bool with_names_,
bool with_types_,
FormatWithNamesAndTypesReader * format_reader_,
DataTypePtr default_type_ = nullptr);
NamesAndTypesList readSchema() override;
protected:
virtual DataTypes readRowAndGetDataTypes() override = 0;
bool with_names;
bool with_types;
private:
FormatWithNamesAndTypesReader * format_reader;
};
}

View File

@ -88,7 +88,7 @@ struct URLBasedDataSourceConfiguration
String url;
String format;
String compression_method = "auto";
String structure;
String structure = "auto";
std::vector<std::pair<String, Field>> headers;
String http_method;

View File

@ -14,7 +14,6 @@
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <IO/WriteHelpers.h>
#include <IO/ReadHelpers.h>
#include <Interpreters/Context.h>
@ -29,6 +28,8 @@
#include <Storages/HDFS/WriteBufferFromHDFS.h>
#include <Storages/PartitionedSink.h>
#include <Formats/ReadSchemaUtils.h>
#include <Formats/FormatFactory.h>
#include <Functions/FunctionsConversion.h>
@ -51,6 +52,69 @@ namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int ACCESS_DENIED;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
namespace
{
/* Recursive directory listing with matched paths as a result.
* Have the same method in StorageFile.
*/
Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match)
{
const size_t first_glob = for_match.find_first_of("*?{");
const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/');
const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/'
const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/'
const size_t next_slash = suffix_with_globs.find('/', 1);
re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash)));
HDFSFileInfo ls;
ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length);
Strings result;
for (int i = 0; i < ls.length; ++i)
{
const String full_path = String(ls.file_info[i].mName);
const size_t last_slash = full_path.rfind('/');
const String file_name = full_path.substr(last_slash);
const bool looking_for_directory = next_slash != std::string::npos;
const bool is_directory = ls.file_info[i].mKind == 'D';
/// Condition with type of current file_info means what kind of path is it in current iteration of ls
if (!is_directory && !looking_for_directory)
{
if (re2::RE2::FullMatch(file_name, matcher))
{
result.push_back(String(ls.file_info[i].mName));
}
}
else if (is_directory && looking_for_directory)
{
if (re2::RE2::FullMatch(file_name, matcher))
{
Strings result_part = LSWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash));
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
}
}
}
return result;
}
std::pair<String, String> getPathFromUriAndUriWithoutPath(const String & uri)
{
const size_t begin_of_path = uri.find('/', uri.find("//") + 2);
return {uri.substr(begin_of_path), uri.substr(0, begin_of_path)};
}
std::vector<String> getPathsList(const String & path_from_uri, const String & uri_without_path, ContextPtr context)
{
HDFSBuilderWrapper builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef());
HDFSFSPtr fs = createHDFSFS(builder.get());
return LSWithRegexpMatching("/", fs, path_from_uri);
}
}
static Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match);
@ -79,12 +143,45 @@ StorageHDFS::StorageHDFS(
checkHDFSURL(uri);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromData(format_name, uri, compression_method, context_);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
ColumnsDescription StorageHDFS::getTableStructureFromData(
const String & format,
const String & uri,
const String & compression_method,
ContextPtr ctx)
{
auto read_buffer_creator = [&]()
{
const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri);
auto paths = getPathsList(path_from_uri, uri, ctx);
if (paths.empty())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files in HDFS with provided path. You must "
"specify table structure manually",
format);
auto compression = chooseCompressionMethod(paths[0], compression_method);
return wrapReadBufferWithCompressionMethod(
std::make_unique<ReadBufferFromHDFS>(uri_without_path, paths[0], ctx->getGlobalContext()->getConfigRef()), compression);
};
return readSchemaFromFormat(format, std::nullopt, read_buffer_creator, ctx);
}
class HDFSSource::DisclosedGlobIterator::Impl
{
public:
@ -339,51 +436,6 @@ private:
};
/* Recursive directory listing with matched paths as a result.
* Have the same method in StorageFile.
*/
Strings listFilesWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match)
{
const size_t first_glob = for_match.find_first_of("*?{");
const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/');
const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/'
const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/'
const size_t next_slash = suffix_with_globs.find('/', 1);
re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash)));
HDFSFileInfo ls;
ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length);
Strings result;
for (int i = 0; i < ls.length; ++i)
{
const String full_path = String(ls.file_info[i].mName);
const size_t last_slash = full_path.rfind('/');
const String file_name = full_path.substr(last_slash);
const bool looking_for_directory = next_slash != std::string::npos;
const bool is_directory = ls.file_info[i].mKind == 'D';
/// Condition with type of current file_info means what kind of path is it in current iteration of ls
if (!is_directory && !looking_for_directory)
{
if (re2::RE2::FullMatch(file_name, matcher))
{
result.push_back(String(ls.file_info[i].mName));
}
}
else if (is_directory && looking_for_directory)
{
if (re2::RE2::FullMatch(file_name, matcher))
{
Strings result_part = listFilesWithRegexpMatching(fs::path(full_path) / "", fs, suffix_with_globs.substr(next_slash));
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
}
}
}
return result;
}
bool StorageHDFS::isColumnOriented() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
@ -400,6 +452,7 @@ Pipe StorageHDFS::read(
{
bool need_path_column = false;
bool need_file_column = false;
for (const auto & column : column_names)
{
if (column == "_path")
@ -528,6 +581,7 @@ void registerStorageHDFS(StorageFactory & factory)
},
{
.supports_sort_order = true, // for partition by
.supports_schema_inference = true,
.source_access_type = AccessType::HDFS,
});
}

View File

@ -31,7 +31,7 @@ public:
size_t max_block_size,
unsigned num_streams) override;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr context) override;
SinkToStoragePtr write(const ASTPtr & query, const StorageMetadataPtr & /*metadata_snapshot*/, ContextPtr /*context*/) override;
void truncate(
const ASTPtr & query,
@ -49,6 +49,12 @@ public:
/// format to read only them. Note: this hack cannot be done with ordinary formats like TSV.
bool isColumnOriented() const;
static ColumnsDescription getTableStructureFromData(
const String & format,
const String & uri,
const String & compression_method,
ContextPtr ctx);
protected:
friend class HDFSSource;
StorageHDFS(

View File

@ -224,7 +224,6 @@ MergeTreeData::MergeTreeData(
{
try
{
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
setProperties(metadata_, metadata_, attach);
if (minmax_idx_date_column_pos == -1)

View File

@ -35,6 +35,7 @@ namespace ErrorCodes
extern const int NO_ELEMENTS_IN_CONFIG;
extern const int UNKNOWN_STORAGE;
extern const int NO_REPLICA_NAME_GIVEN;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
@ -258,6 +259,34 @@ If you use the Replicated version of engines, see https://clickhouse.com/docs/en
return help;
}
static ColumnsDescription getColumnsDescriptionFromZookeeper(const String & raw_zookeeper_path, ContextMutablePtr context)
{
String zookeeper_name = zkutil::extractZooKeeperName(raw_zookeeper_path);
String zookeeper_path = zkutil::extractZooKeeperPath(raw_zookeeper_path, true);
if (!context->hasZooKeeper() && !context->hasAuxiliaryZooKeeper(zookeeper_name))
throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure without zookeeper, you must specify the structure manually"};
zkutil::ZooKeeperPtr zookeeper;
try
{
if (zookeeper_name == StorageReplicatedMergeTree::getDefaultZooKeeperName())
zookeeper = context->getZooKeeper();
else
zookeeper = context->getAuxiliaryZooKeeper(zookeeper_name);
}
catch (...)
{
throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure from zookeeper, because cannot get zookeeper: {}. You must specify structure manually", getCurrentExceptionMessage(false)};
}
if (!zookeeper->exists(zookeeper_path + "/replicas"))
throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "Cannot get replica structure, because there no other replicas in zookeeper. You must specify the structure manually"};
Coordination::Stat columns_stat;
return ColumnsDescription::parse(zookeeper->get(fs::path(zookeeper_path) / "columns", &columns_stat));
}
static StoragePtr create(const StorageFactory::Arguments & args)
{
@ -638,7 +667,14 @@ static StoragePtr create(const StorageFactory::Arguments & args)
String date_column_name;
StorageInMemoryMetadata metadata;
metadata.setColumns(args.columns);
ColumnsDescription columns;
if (args.columns.empty() && replicated)
columns = getColumnsDescriptionFromZookeeper(zookeeper_path, args.getContext());
else
columns = args.columns;
metadata.setColumns(columns);
metadata.setComment(args.comment);
std::unique_ptr<MergeTreeSettings> storage_settings;
@ -705,12 +741,12 @@ static StoragePtr create(const StorageFactory::Arguments & args)
if (args.query.columns_list && args.query.columns_list->indices)
for (auto & index : args.query.columns_list->indices->children)
metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, args.columns, args.getContext()));
metadata.secondary_indices.push_back(IndexDescription::getIndexFromAST(index, columns, args.getContext()));
if (args.query.columns_list && args.query.columns_list->projections)
for (auto & projection_ast : args.query.columns_list->projections->children)
{
auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, args.columns, args.getContext());
auto projection = ProjectionDescription::getProjectionFromAST(projection_ast, columns, args.getContext());
metadata.projections.add(std::move(projection));
}
@ -720,10 +756,10 @@ static StoragePtr create(const StorageFactory::Arguments & args)
constraints.push_back(constraint);
metadata.constraints = ConstraintsDescription(constraints);
auto column_ttl_asts = args.columns.getColumnTTLs();
auto column_ttl_asts = columns.getColumnTTLs();
for (const auto & [name, ast] : column_ttl_asts)
{
auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, args.columns, args.getContext(), metadata.primary_key);
auto new_ttl_entry = TTLDescription::getTTLFromAST(ast, columns, args.getContext(), metadata.primary_key);
metadata.column_ttls_by_name[name] = new_ttl_entry;
}
@ -850,6 +886,7 @@ void registerStorageMergeTree(StorageFactory & factory)
features.supports_replication = true;
features.supports_deduplication = true;
features.supports_schema_inference = true;
factory.registerStorage("ReplicatedMergeTree", create, features);
factory.registerStorage("ReplicatedCollapsingMergeTree", create, features);

View File

@ -126,6 +126,12 @@ StorageBuffer::StorageBuffer(
, bg_pool(getContext()->getBufferFlushSchedulePool())
{
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto dest_table = DatabaseCatalog::instance().getTable(destination_id, context_);
storage_metadata.setColumns(dest_table->getInMemoryMetadataPtr()->getColumns());
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
@ -1167,6 +1173,7 @@ void registerStorageBuffer(StorageFactory & factory)
},
{
.supports_parallel_insert = true,
.supports_schema_inference = true,
});
}

View File

@ -13,6 +13,7 @@
#include <Storages/Distributed/DistributedSink.h>
#include <Storages/StorageFactory.h>
#include <Storages/AlterCommands.h>
#include <Storages/getStructureOfRemoteTable.h>
#include <Columns/ColumnConst.h>
@ -24,7 +25,6 @@
#include <Common/randomSeed.h>
#include <Common/formatReadable.h>
#include <Parsers/ASTDropQuery.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/ASTIdentifier.h>
@ -42,7 +42,6 @@
#include <Interpreters/ClusterProxy/executeQuery.h>
#include <Interpreters/Cluster.h>
#include <Interpreters/ExpressionAnalyzer.h>
#include <Interpreters/InterpreterAlterQuery.h>
#include <Interpreters/InterpreterDescribeQuery.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/JoinedTables.h>
@ -63,7 +62,6 @@
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Sinks/EmptySink.h>
#include <Core/Field.h>
#include <Core/Settings.h>
#include <IO/ReadHelpers.h>
@ -71,8 +69,6 @@
#include <IO/Operators.h>
#include <IO/ConnectionTimeoutsContext.h>
#include <Poco/DirectoryIterator.h>
#include <memory>
#include <filesystem>
#include <optional>
@ -329,7 +325,16 @@ StorageDistributed::StorageDistributed(
, rng(randomSeed())
{
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
StorageID id = StorageID::createEmpty();
id.table_name = remote_table;
id.database_name = remote_database;
storage_metadata.setColumns(getStructureOfRemoteTable(*getCluster(), id, getContext(), remote_table_function_ptr));
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
@ -1398,6 +1403,7 @@ void registerStorageDistributed(StorageFactory & factory)
{
.supports_settings = true,
.supports_parallel_insert = true,
.supports_schema_inference = true,
.source_access_type = AccessType::REMOTE,
});
}

View File

@ -66,6 +66,7 @@ public:
bool supports_deduplication = false;
/// See also IStorage::supportsParallelInsert()
bool supports_parallel_insert = false;
bool supports_schema_inference = false;
AccessType source_access_type = AccessType::NONE;
};
@ -98,6 +99,7 @@ public:
.supports_replication = false,
.supports_deduplication = false,
.supports_parallel_insert = false,
.supports_schema_inference = false,
.source_access_type = AccessType::NONE,
});
@ -126,6 +128,12 @@ public:
AccessType getSourceAccessType(const String & table_engine) const;
bool checkIfStorageSupportsSchemaInterface(const String & storage_name)
{
if (storages.contains(storage_name))
return storages[storage_name].features.supports_schema_inference;
return false;
}
private:
Storages storages;
};

View File

@ -15,8 +15,9 @@
#include <IO/WriteBufferFromFile.h>
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <DataTypes/DataTypeString.h>
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Processors/Sinks/SinkToStorage.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
@ -38,6 +39,7 @@
#include <Processors/Sources/SourceWithProgress.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/ISchemaReader.h>
#include <Processors/Sources/NullSource.h>
#include <QueryPipeline/Pipe.h>
#include <Processors/Executors/PullingPipelineExecutor.h>
@ -63,6 +65,7 @@ namespace ErrorCodes
extern const int INCOMPATIBLE_COLUMNS;
extern const int CANNOT_STAT;
extern const int LOGICAL_ERROR;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
namespace
@ -135,6 +138,56 @@ void checkCreationIsAllowed(ContextPtr context_global, const std::string & db_di
throw Exception("File must not be a directory", ErrorCodes::INCORRECT_FILE_NAME);
}
std::unique_ptr<ReadBuffer> createReadBuffer(
const String & current_path,
bool use_table_fd,
const String & storage_name,
int table_fd,
const String & compression_method,
ContextPtr context)
{
std::unique_ptr<ReadBuffer> nested_buffer;
CompressionMethod method;
struct stat file_stat{};
if (use_table_fd)
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != fstat(table_fd, &file_stat))
throwFromErrno("Cannot stat table file descriptor, inside " + storage_name, ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFileDescriptorPRead>(table_fd);
else
nested_buffer = std::make_unique<ReadBufferFromFileDescriptor>(table_fd);
method = chooseCompressionMethod("", compression_method);
}
else
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != stat(current_path.c_str(), &file_stat))
throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFilePRead>(current_path, context->getSettingsRef().max_read_buffer_size);
else
nested_buffer = std::make_unique<ReadBufferFromFile>(current_path, context->getSettingsRef().max_read_buffer_size);
method = chooseCompressionMethod(current_path, compression_method);
}
/// For clickhouse-local add progress callback to display progress bar.
if (context->getApplicationType() == Context::ApplicationType::LOCAL)
{
auto & in = static_cast<ReadBufferFromFileDescriptor &>(*nested_buffer);
in.setProgressCallback(context);
}
return wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method);
}
}
Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read)
@ -164,6 +217,42 @@ Strings StorageFile::getPathsList(const String & table_path, const String & user
return paths;
}
ColumnsDescription StorageFile::getTableStructureFromData(
const String & format,
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
ContextPtr context)
{
if (format == "Distributed")
{
if (paths.empty())
throw Exception(
"Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME);
auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]);
return ColumnsDescription(source->getOutputs().front().getHeader().getNamesAndTypesList());
}
auto read_buffer_creator = [&]()
{
String path;
auto it = std::find_if(paths.begin(), paths.end(), [](const String & p){ return std::filesystem::exists(p); });
if (it == paths.end())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path. You must specify "
"table structure manually",
format);
path = *it;
return createReadBuffer(path, false, "File", -1, compression_method, context);
};
return readSchemaFromFormat(format, format_settings, read_buffer_creator, context);
}
bool StorageFile::isColumnOriented() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
@ -182,10 +271,13 @@ StorageFile::StorageFile(int table_fd_, CommonArguments args)
throw Exception("Using file descriptor as source of storage isn't allowed for server daemons", ErrorCodes::DATABASE_ACCESS_DENIED);
if (args.format_name == "Distributed")
throw Exception("Distributed format is allowed only with explicit file path", ErrorCodes::INCORRECT_FILE_NAME);
if (args.columns.empty())
throw Exception("Automatic schema inference is not allowed when using file descriptor as source of storage", ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE);
is_db_table = false;
use_table_fd = true;
table_fd = table_fd_;
setStorageMetadata(args);
}
StorageFile::StorageFile(const std::string & table_path_, const std::string & user_files_path, CommonArguments args)
@ -194,22 +286,7 @@ StorageFile::StorageFile(const std::string & table_path_, const std::string & us
is_db_table = false;
paths = getPathsList(table_path_, user_files_path, args.getContext(), total_bytes_to_read);
path_for_partitioned_write = table_path_;
if (args.format_name == "Distributed")
{
if (paths.empty())
throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME);
auto & first_path = paths[0];
Block header = StorageDistributedDirectoryMonitor::createSourceFromFile(first_path)->getOutputs().front().getHeader();
StorageInMemoryMetadata storage_metadata;
auto columns = ColumnsDescription(header.getNamesAndTypesList());
if (!args.columns.empty() && columns != args.columns)
throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS);
storage_metadata.setColumns(columns);
setInMemoryMetadata(storage_metadata);
}
setStorageMetadata(args);
}
StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArguments args)
@ -225,6 +302,8 @@ StorageFile::StorageFile(const std::string & relative_table_dir_path, CommonArgu
paths = {getTablePath(table_dir_path, format_name)};
if (fs::exists(paths[0]))
total_bytes_to_read = fs::file_size(paths[0]);
setStorageMetadata(args);
}
StorageFile::StorageFile(CommonArguments args)
@ -233,9 +312,21 @@ StorageFile::StorageFile(CommonArguments args)
, format_settings(args.format_settings)
, compression_method(args.compression_method)
, base_path(args.getContext()->getPath())
{
}
void StorageFile::setStorageMetadata(CommonArguments args)
{
StorageInMemoryMetadata storage_metadata;
if (args.format_name != "Distributed")
if (args.format_name == "Distributed" || args.columns.empty())
{
auto columns = getTableStructureFromData(format_name, paths, compression_method, format_settings, args.getContext());
if (!args.columns.empty() && args.columns != columns)
throw Exception("Table structure and file structure are different", ErrorCodes::INCOMPATIBLE_COLUMNS);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(args.columns);
storage_metadata.setConstraints(args.constraints);
@ -350,46 +441,7 @@ public:
}
}
std::unique_ptr<ReadBuffer> nested_buffer;
CompressionMethod method;
struct stat file_stat{};
if (storage->use_table_fd)
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != fstat(storage->table_fd, &file_stat))
throwFromErrno("Cannot stat table file descriptor, inside " + storage->getName(), ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFileDescriptorPRead>(storage->table_fd);
else
nested_buffer = std::make_unique<ReadBufferFromFileDescriptor>(storage->table_fd);
method = chooseCompressionMethod("", storage->compression_method);
}
else
{
/// Check if file descriptor allows random reads (and reading it twice).
if (0 != stat(current_path.c_str(), &file_stat))
throwFromErrno("Cannot stat file " + current_path, ErrorCodes::CANNOT_STAT);
if (S_ISREG(file_stat.st_mode))
nested_buffer = std::make_unique<ReadBufferFromFilePRead>(current_path, context->getSettingsRef().max_read_buffer_size);
else
nested_buffer = std::make_unique<ReadBufferFromFile>(current_path, context->getSettingsRef().max_read_buffer_size);
method = chooseCompressionMethod(current_path, storage->compression_method);
}
/// For clickhouse-local add progress callback to display progress bar.
if (context->getApplicationType() == Context::ApplicationType::LOCAL)
{
auto & in = static_cast<ReadBufferFromFileDescriptor &>(*nested_buffer);
in.setProgressCallback(context);
}
read_buf = wrapReadBufferWithCompressionMethod(std::move(nested_buffer), method);
read_buf = createReadBuffer(current_path, storage->use_table_fd, storage->getName(), storage->table_fd, storage->compression_method, context);
auto get_block_for_format = [&]() -> Block
{
@ -853,7 +905,8 @@ void registerStorageFile(StorageFactory & factory)
{
StorageFactory::StorageFeatures storage_features{
.supports_settings = true,
.source_access_type = AccessType::FILE
.supports_schema_inference = true,
.source_access_type = AccessType::FILE,
};
factory.registerStorage(

View File

@ -1,6 +1,7 @@
#pragma once
#include <Storages/IStorage.h>
#include <base/logger_useful.h>
#include <atomic>
@ -70,6 +71,13 @@ public:
bool supportsPartitionBy() const override { return true; }
static ColumnsDescription getTableStructureFromData(
const String & format,
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
ContextPtr context);
protected:
friend class StorageFileSource;
friend class StorageFileSink;
@ -86,6 +94,8 @@ protected:
private:
explicit StorageFile(CommonArguments args);
void setStorageMetadata(CommonArguments args);
std::string format_name;
// We use format settings from global context + CREATE query for File table
// function -- in this case, format_settings is set.

View File

@ -43,6 +43,7 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int SAMPLING_NOT_SUPPORTED;
extern const int ALTER_OF_COLUMN_IS_FORBIDDEN;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
StorageMerge::StorageMerge(
@ -61,7 +62,7 @@ StorageMerge::StorageMerge(
, database_is_regexp(database_is_regexp_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
@ -82,11 +83,19 @@ StorageMerge::StorageMerge(
, database_is_regexp(database_is_regexp_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setColumns(columns_.empty() ? getColumnsDescriptionFromSourceTables() : columns_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
ColumnsDescription StorageMerge::getColumnsDescriptionFromSourceTables() const
{
auto table = getFirstTable([](auto && t) { return t; });
if (!table)
throw Exception{ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "There are no tables satisfied provided regexp, you must specify table structure manually"};
return table->getInMemoryMetadataPtr()->getColumns();
}
template <typename F>
StoragePtr StorageMerge::getFirstTable(F && predicate) const
{
@ -762,7 +771,6 @@ void StorageMerge::convertingSourceStream(
IStorage::ColumnSizeByName StorageMerge::getColumnSizes() const
{
auto first_materialized_mysql = getFirstTable([](const StoragePtr & table) { return table && table->getName() == "MaterializedMySQL"; });
if (!first_materialized_mysql)
return {};
@ -816,6 +824,9 @@ void registerStorageMerge(StorageFactory & factory)
return StorageMerge::create(
args.table_id, args.columns, args.comment, source_database_name_or_regexp, is_regexp, table_name_regexp, args.getContext());
},
{
.supports_schema_inference = true
});
}

View File

@ -132,6 +132,8 @@ protected:
static SelectQueryInfo getModifiedQueryInfo(
const SelectQueryInfo & query_info, ContextPtr modified_context, const StorageID & current_storage_id, bool is_merge_engine);
ColumnsDescription getColumnsDescriptionFromSourceTables() const;
};
}

View File

@ -3,7 +3,6 @@
#include "Common/hex.h"
#include <Common/Macros.h>
#include <Common/StringUtils/StringUtils.h>
#include <Common/ThreadPool.h>
#include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h>
#include <Common/escapeForFileName.h>
@ -20,7 +19,6 @@
#include <Storages/MergeTree/MergeTreeBackgroundExecutor.h>
#include <Storages/MergeTree/MergedBlockOutputStream.h>
#include <Storages/MergeTree/PinnedPartUUIDs.h>
#include <Storages/MergeTree/PartitionPruner.h>
#include <Storages/MergeTree/ReplicatedMergeTreeTableMetadata.h>
#include <Storages/MergeTree/ReplicatedMergeTreeSink.h>
#include <Storages/MergeTree/ReplicatedMergeTreeQuorumEntry.h>
@ -35,7 +33,6 @@
#include <Storages/MergeTree/LeaderElection.h>
#include <Databases/IDatabase.h>
#include <Databases/DatabaseOnDisk.h>
#include <Parsers/formatAST.h>
@ -45,7 +42,6 @@
#include <Parsers/ASTLiteral.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTCheckQuery.h>
#include <Parsers/ASTSetQuery.h>
#include <Processors/QueryPlan/QueryPlan.h>
#include <Processors/Sources/RemoteSource.h>
@ -68,7 +64,6 @@
#include <Poco/DirectoryIterator.h>
#include <base/range.h>
#include <base/scope_guard.h>
#include <base/scope_guard_safe.h>
@ -194,56 +189,6 @@ zkutil::ZooKeeperPtr StorageReplicatedMergeTree::getZooKeeper() const
return res;
}
static std::string normalizeZooKeeperPath(std::string zookeeper_path, bool check_starts_with_slash, Poco::Logger * log = nullptr)
{
if (!zookeeper_path.empty() && zookeeper_path.back() == '/')
zookeeper_path.resize(zookeeper_path.size() - 1);
/// If zookeeper chroot prefix is used, path should start with '/', because chroot concatenates without it.
if (!zookeeper_path.empty() && zookeeper_path.front() != '/')
{
/// Do not allow this for new tables, print warning for tables created in old versions
if (check_starts_with_slash)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "ZooKeeper path must starts with '/', got '{}'", zookeeper_path);
if (log)
LOG_WARNING(log, "ZooKeeper path ('{}') does not start with '/'. It will not be supported in future releases");
zookeeper_path = "/" + zookeeper_path;
}
return zookeeper_path;
}
static String extractZooKeeperName(const String & path)
{
static constexpr auto default_zookeeper_name = "default";
if (path.empty())
throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS);
if (path[0] == '/')
return default_zookeeper_name;
auto pos = path.find(":/");
if (pos != String::npos && pos < path.find('/'))
{
auto zookeeper_name = path.substr(0, pos);
if (zookeeper_name.empty())
throw Exception("Zookeeper path should start with '/' or '<auxiliary_zookeeper_name>:/'", ErrorCodes::BAD_ARGUMENTS);
return zookeeper_name;
}
return default_zookeeper_name;
}
static String extractZooKeeperPath(const String & path, bool check_starts_with_slash, Poco::Logger * log = nullptr)
{
if (path.empty())
throw Exception("ZooKeeper path should not be empty", ErrorCodes::BAD_ARGUMENTS);
if (path[0] == '/')
return normalizeZooKeeperPath(path, check_starts_with_slash, log);
auto pos = path.find(":/");
if (pos != String::npos && pos < path.find('/'))
{
return normalizeZooKeeperPath(path.substr(pos + 1, String::npos), check_starts_with_slash, log);
}
return normalizeZooKeeperPath(path, check_starts_with_slash, log);
}
static MergeTreePartInfo makeDummyDropRangeForMovePartitionOrAttachPartitionFrom(const String & partition_id)
{
/// NOTE We don't have special log entry type for MOVE PARTITION/ATTACH PARTITION FROM,
@ -287,8 +232,8 @@ StorageReplicatedMergeTree::StorageReplicatedMergeTree(
true, /// require_part_metadata
attach,
[this] (const std::string & name) { enqueuePartForCheck(name); })
, zookeeper_name(extractZooKeeperName(zookeeper_path_))
, zookeeper_path(extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log))
, zookeeper_name(zkutil::extractZooKeeperName(zookeeper_path_))
, zookeeper_path(zkutil::extractZooKeeperPath(zookeeper_path_, /* check_starts_with_slash */ !attach, log))
, replica_name(replica_name_)
, replica_path(fs::path(zookeeper_path) / "replicas" / replica_name_)
, reader(*this)
@ -5561,8 +5506,8 @@ void StorageReplicatedMergeTree::fetchPartition(
info.table_id = getStorageID();
info.table_id.uuid = UUIDHelpers::Nil;
auto expand_from = query_context->getMacros()->expand(from_, info);
String auxiliary_zookeeper_name = extractZooKeeperName(expand_from);
String from = extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true);
String auxiliary_zookeeper_name = zkutil::extractZooKeeperName(expand_from);
String from = zkutil::extractZooKeeperPath(expand_from, /* check_starts_with_slash */ true);
if (from.empty())
throw Exception("ZooKeeper path should not be empty", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
@ -6638,7 +6583,7 @@ void StorageReplicatedMergeTree::movePartitionToShard(
if (!move_part)
throw Exception("MOVE PARTITION TO SHARD is not supported, use MOVE PART instead", ErrorCodes::NOT_IMPLEMENTED);
if (normalizeZooKeeperPath(zookeeper_path, /* check_starts_with_slash */ true) == normalizeZooKeeperPath(to, /* check_starts_with_slash */ true))
if (zkutil::normalizeZooKeeperPath(zookeeper_path, /* check_starts_with_slash */ true) == zkutil::normalizeZooKeeperPath(to, /* check_starts_with_slash */ true))
throw Exception("Source and destination are the same", ErrorCodes::BAD_ARGUMENTS);
auto zookeeper = getZooKeeper();

View File

@ -263,6 +263,8 @@ public:
bool createEmptyPartInsteadOfLost(zkutil::ZooKeeperPtr zookeeper, const String & lost_part_name);
static const String getDefaultZooKeeperName() { return default_zookeeper_name; }
private:
std::atomic_bool are_restoring_replica {false};

View File

@ -28,6 +28,7 @@
#include <IO/WriteHelpers.h>
#include <Formats/FormatFactory.h>
#include <Formats/ReadSchemaUtils.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
#include <Processors/Formats/IOutputFormat.h>
@ -70,6 +71,7 @@ namespace ErrorCodes
extern const int S3_ERROR;
extern const int UNEXPECTED_EXPRESSION;
extern const int CANNOT_OPEN_FILE;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
}
class IOutputFormat;
@ -480,13 +482,39 @@ StorageS3::StorageS3(
{
context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri);
StorageInMemoryMetadata storage_metadata;
updateClientAndAuthSettings(context_, client_auth);
if (columns_.empty())
{
auto columns = getTableStructureFromDataImpl(format_name, client_auth, max_single_read_retries_, compression_method, distributed_processing_, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
updateClientAndAuthSettings(context_, client_auth);
}
std::shared_ptr<StorageS3Source::IteratorWrapper> StorageS3::createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context)
{
std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper{nullptr};
if (distributed_processing)
{
return std::make_shared<StorageS3Source::IteratorWrapper>(
[callback = local_context->getReadTaskCallback()]() -> String {
return callback();
});
}
/// Iterate through disclosed globs and make a source for each file
auto glob_iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
return std::make_shared<StorageS3Source::IteratorWrapper>([glob_iterator]()
{
return glob_iterator->next();
});
}
Pipe StorageS3::read(
const Names & column_names,
@ -510,23 +538,7 @@ Pipe StorageS3::read(
need_file_column = true;
}
std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper{nullptr};
if (distributed_processing)
{
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>(
[callback = local_context->getReadTaskCallback()]() -> String {
return callback();
});
}
else
{
/// Iterate through disclosed globs and make a source for each file
auto glob_iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>([glob_iterator]()
{
return glob_iterator->next();
});
}
std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper = createFileIterator(client_auth, distributed_processing, local_context);
for (size_t i = 0; i < num_streams; ++i)
{
@ -707,6 +719,51 @@ StorageS3Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPt
return configuration;
}
ColumnsDescription StorageS3::getTableStructureFromData(
const String & format,
const S3::URI & uri,
const String & access_key_id,
const String & secret_access_key,
UInt64 max_connections,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
{
ClientAuthentication client_auth{uri, access_key_id, secret_access_key, max_connections, {}, {}};
updateClientAndAuthSettings(ctx, client_auth);
return getTableStructureFromDataImpl(format, client_auth, max_single_read_retries, compression_method, distributed_processing, format_settings, ctx);
}
ColumnsDescription StorageS3::getTableStructureFromDataImpl(
const String & format,
const ClientAuthentication & client_auth,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
{
auto read_buffer_creator = [&]()
{
auto file_iterator = createFileIterator(client_auth, distributed_processing, ctx);
String current_key = (*file_iterator)();
if (current_key.empty())
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path in S3. You must specify "
"table structure manually",
format);
return wrapReadBufferWithCompressionMethod(
std::make_unique<ReadBufferFromS3>(client_auth.client, client_auth.uri.bucket, current_key, max_single_read_retries, ctx->getReadSettings(), DBMS_DEFAULT_BUFFER_SIZE),
chooseCompressionMethod(current_key, compression_method));
};
return readSchemaFromFormat(format, format_settings, read_buffer_creator, ctx);
}
void registerStorageS3Impl(const String & name, StorageFactory & factory)
{

View File

@ -147,8 +147,19 @@ public:
static StorageS3Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
private:
static ColumnsDescription getTableStructureFromData(
const String & format,
const S3::URI & uri,
const String & access_key_id,
const String & secret_access_key,
UInt64 max_connections,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
private:
friend class StorageS3Cluster;
friend class TableFunctionS3Cluster;
@ -175,6 +186,17 @@ private:
ASTPtr partition_by;
static void updateClientAndAuthSettings(ContextPtr, ClientAuthentication &);
static std::shared_ptr<StorageS3Source::IteratorWrapper> createFileIterator(const ClientAuthentication & client_auth, bool distributed_processing, ContextPtr local_context);
static ColumnsDescription getTableStructureFromDataImpl(
const String & format,
const ClientAuthentication & client_auth,
UInt64 max_single_read_retries,
const String & compression_method,
bool distributed_processing,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
};
}

View File

@ -13,8 +13,9 @@
#include <IO/ConnectionTimeoutsContext.h>
#include <Formats/FormatFactory.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Formats/ReadSchemaUtils.h>
#include <Processors/Formats/IInputFormat.h>
#include <Processors/Formats/IOutputFormat.h>
#include <Common/parseRemoteDescription.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
@ -40,7 +41,7 @@ namespace ErrorCodes
IStorageURLBase::IStorageURLBase(
const String & uri_,
ContextPtr /*context_*/,
ContextPtr context_,
const StorageID & table_id_,
const String & format_name_,
const std::optional<FormatSettings> & format_settings_,
@ -61,12 +62,48 @@ IStorageURLBase::IStorageURLBase(
, partition_by(partition_by_)
{
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromData(format_name, uri, compression_method, headers, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
storage_metadata.setComment(comment);
setInMemoryMetadata(storage_metadata);
}
ColumnsDescription IStorageURLBase::getTableStructureFromData(
const String & format,
const String & uri,
const String & compression_method,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
ContextPtr context)
{
auto read_buffer_creator = [&]()
{
auto parsed_uri = Poco::URI(uri);
return wrapReadBufferWithCompressionMethod(
std::make_unique<ReadWriteBufferFromHTTP>(
parsed_uri,
Poco::Net::HTTPRequest::HTTP_GET,
nullptr,
ConnectionTimeouts::getHTTPTimeouts(context),
Poco::Net::HTTPBasicCredentials{},
context->getSettingsRef().max_http_get_redirects,
DBMS_DEFAULT_BUFFER_SIZE,
context->getReadSettings(),
headers,
ReadWriteBufferFromHTTP::Range{},
context->getRemoteHostFilter()),
chooseCompressionMethod(parsed_uri.getPath(), compression_method));
};
return readSchemaFromFormat(format, format_settings, read_buffer_creator, context);
}
namespace
{
ReadWriteBufferFromHTTP::HTTPHeaderEntries getHeaders(
@ -642,6 +679,7 @@ void registerStorageURL(StorageFactory & factory)
},
{
.supports_settings = true,
.supports_schema_inference = true,
.source_access_type = AccessType::URL,
});
}

View File

@ -41,6 +41,14 @@ public:
bool supportsPartitionBy() const override { return true; }
static ColumnsDescription getTableStructureFromData(
const String & format,
const String & uri,
const String & compression_method,
const ReadWriteBufferFromHTTP::HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
ContextPtr context);
protected:
IStorageURLBase(
const String & uri_,

View File

@ -15,25 +15,23 @@ namespace DB
{
StoragePtr ITableFunction::execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name,
ColumnsDescription cached_columns) const
ColumnsDescription cached_columns, bool use_global_context) const
{
ProfileEvents::increment(ProfileEvents::TableFunctionExecute);
context->checkAccess(AccessType::CREATE_TEMPORARY_TABLE | StorageFactory::instance().getSourceAccessType(getStorageTypeName()));
auto context_to_use = use_global_context ? context->getGlobalContext() : context;
if (cached_columns.empty())
return executeImpl(ast_function, context, table_name, std::move(cached_columns));
/// We have table structure, so it's CREATE AS table_function().
/// We should use global context here because there will be no query context on server startup
/// and because storage lifetime is bigger than query context lifetime.
auto global_context = context->getGlobalContext();
if (hasStaticStructure() && cached_columns == getActualTableStructure(context))
return executeImpl(ast_function, global_context, table_name, std::move(cached_columns));
return executeImpl(ast_function, context_to_use, table_name, std::move(cached_columns));
auto this_table_function = shared_from_this();
auto get_storage = [=]() -> StoragePtr
{
return this_table_function->executeImpl(ast_function, global_context, table_name, cached_columns);
return this_table_function->executeImpl(ast_function, context_to_use, table_name, cached_columns);
};
/// It will request actual table structure and create underlying storage lazily

View File

@ -54,7 +54,7 @@ public:
/// Create storage according to the query.
StoragePtr
execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}) const;
execute(const ASTPtr & ast_function, ContextPtr context, const std::string & table_name, ColumnsDescription cached_columns_ = {}, bool use_global_context = false) const;
virtual ~ITableFunction() = default;

View File

@ -1,4 +1,3 @@
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/ITableFunctionFileLike.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
@ -6,16 +5,16 @@
#include <Parsers/ASTLiteral.h>
#include <Common/Exception.h>
#include <Common/typeid_cast.h>
#include <Storages/StorageFile.h>
#include <Storages/Distributed/DirectoryMonitor.h>
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Processors/ISource.h>
#include <Formats/FormatFactory.h>
namespace DB
{
@ -23,10 +22,27 @@ namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int INCORRECT_FILE_NAME;
extern const int BAD_ARGUMENTS;
}
namespace
{
void checkIfFormatSupportsAutoStructure(const String & name, const String & format)
{
if (name == "file" && format == "Distributed")
return;
if (FormatFactory::instance().checkIfFormatHasAnySchemaReader(format))
return;
throw Exception(
"Table function '" + name
+ "' allows automatic structure determination only for formats that support schema inference and for Distributed format in table function "
"'file'",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
}
void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
/// Parse args
@ -46,21 +62,23 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() == 2 && getName() == "file")
if (args.size() == 2)
{
if (format == "Distributed")
checkIfFormatSupportsAutoStructure(getName(), format);
return;
throw Exception("Table function '" + getName() + "' allows 2 arguments only for Distributed format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
}
if (args.size() != 3 && args.size() != 4)
throw Exception("Table function '" + getName() + "' requires 3 or 4 arguments: filename, format, structure and compression method (default auto).",
throw Exception("Table function '" + getName() + "' requires 2, 3 or 4 arguments: filename, format, structure (default auto) and compression method (default auto)",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
if (structure == "auto")
checkIfFormatSupportsAutoStructure(getName(), format);
if (structure.empty())
throw Exception(ErrorCodes::BAD_ARGUMENTS,
"Table structure is empty for table function '{}'",
"Table structure is empty for table function '{}'. If you want to use automatic schema inference, use 'auto'",
ast_function->formatForErrorMessage());
if (args.size() == 4)
@ -69,25 +87,12 @@ void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, Context
StoragePtr ITableFunctionFileLike::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
auto columns = getActualTableStructure(context);
ColumnsDescription columns;
if (structure != "auto")
columns = parseColumnsListFromString(structure, context);
StoragePtr storage = getStorage(filename, format, columns, context, table_name, compression_method);
storage->startup();
return storage;
}
ColumnsDescription ITableFunctionFileLike::getActualTableStructure(ContextPtr context) const
{
if (structure.empty())
{
assert(getName() == "file" && format == "Distributed");
size_t total_bytes_to_read = 0;
Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read);
if (paths.empty())
throw Exception("Cannot get table structure from file, because no files match specified name", ErrorCodes::INCORRECT_FILE_NAME);
auto source = StorageDistributedDirectoryMonitor::createSourceFromFile(paths[0]);
return ColumnsDescription{source->getOutputs().front().getHeader().getNamesAndTypesList()};
}
return parseColumnsListFromString(structure, context);
}
}

View File

@ -8,7 +8,7 @@ class ColumnsDescription;
class Context;
/*
* function(source, format, structure) - creates a temporary storage from formatted source
* function(source, format, structure[, compression_method]) - creates a temporary storage from formatted source
*/
class ITableFunctionFileLike : public ITableFunction
{
@ -18,7 +18,7 @@ protected:
String filename;
String format;
String structure;
String structure = "auto";
String compression_method = "auto";
private:
@ -28,8 +28,7 @@ private:
const String & source, const String & format, const ColumnsDescription & columns, ContextPtr global_context,
const std::string & table_name, const String & compression_method) const = 0;
ColumnsDescription getActualTableStructure(ContextPtr context) const override;
bool hasStaticStructure() const override { return true; }
bool hasStaticStructure() const override { return structure != "auto"; }
};
}

View File

@ -1,4 +1,5 @@
#include <TableFunctions/TableFunctionFile.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include "registerTableFunctions.h"
#include <Access/Common/AccessFlags.h>
@ -9,11 +10,13 @@
namespace DB
{
StoragePtr TableFunctionFile::getStorage(const String & source,
const String & format_, const ColumnsDescription & columns,
ContextPtr global_context, const std::string & table_name,
const std::string & compression_method_) const
{
LOG_DEBUG(&Poco::Logger::get("TableFunctionFile"), "getStorage");
// For `file` table function, we are going to use format settings from the
// query context.
StorageFile::CommonArguments args{
@ -30,8 +33,21 @@ StoragePtr TableFunctionFile::getStorage(const String & source,
return StorageFile::create(source, global_context->getUserFilesPath(), args);
}
ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context) const
{
if (structure == "auto")
{
size_t total_bytes_to_read = 0;
Strings paths = StorageFile::getPathsList(filename, context->getUserFilesPath(), context, total_bytes_to_read);
return StorageFile::getTableStructureFromData(format, paths, compression_method, std::nullopt, context);
}
return parseColumnsListFromString(structure, context);
}
void registerTableFunctionFile(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionFile>();
}
}

Some files were not shown because too many files have changed in this diff Show More