mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 23:21:59 +00:00
Implement schema inference for most input formats
This commit is contained in:
parent
e671252131
commit
8112a71233
@ -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)
|
||||
{
|
||||
|
@ -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) \
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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.
|
||||
|
||||
|
@ -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)
|
||||
|
@ -76,10 +76,16 @@ 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)
|
||||
throw Exception("Missing definition of columns.", ErrorCodes::EMPTY_LIST_OF_COLUMNS_PASSED);
|
||||
|
||||
columns = InterpreterCreateQuery::getColumnsDescription(*ast_create_query.columns_list->columns, context, true);
|
||||
constraints = InterpreterCreateQuery::getConstraintsDescription(ast_create_query.columns_list->constraints);
|
||||
{
|
||||
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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
readQuotedString(result, buf);
|
||||
if constexpr (read_string)
|
||||
readQuotedString(result, buf);
|
||||
else
|
||||
readQuotedFieldIntoString(result, buf);
|
||||
break;
|
||||
case FormatSettings::EscapingRule::JSON:
|
||||
readJSONString(result, buf);
|
||||
if constexpr (read_string)
|
||||
readJSONString(result, buf);
|
||||
else
|
||||
readJSONFieldIntoString(result, buf);
|
||||
break;
|
||||
case FormatSettings::EscapingRule::Raw:
|
||||
readString(result, buf);
|
||||
break;
|
||||
case FormatSettings::EscapingRule::CSV:
|
||||
readCSVString(result, buf, format_settings.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;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
112
src/Formats/ReadSchemaUtils.cpp
Normal file
112
src/Formats/ReadSchemaUtils.cpp
Normal 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);
|
||||
}
|
||||
|
||||
}
|
30
src/Formats/ReadSchemaUtils.h
Normal file
30
src/Formats/ReadSchemaUtils.h
Normal 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);
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
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;
|
||||
auto parse_func = [](ReadBuffer & in)
|
||||
{
|
||||
Float64 tmp;
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
@ -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());
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
|
@ -557,34 +557,43 @@ 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;
|
||||
|
||||
if (!select_p.parse(pos, select, expected)) /// AS SELECT ...
|
||||
/// CREATE|ATTACH TABLE ... AS ...
|
||||
if (s_as.ignore(pos, expected))
|
||||
{
|
||||
/// ENGINE can not be specified for table functions.
|
||||
if (storage || !table_function_p.parse(pos, as_table_function, expected))
|
||||
if (!select_p.parse(pos, select, expected)) /// AS SELECT ...
|
||||
{
|
||||
/// AS [db.]table
|
||||
if (!name_p.parse(pos, as_table, expected))
|
||||
return false;
|
||||
|
||||
if (s_dot.ignore(pos, expected))
|
||||
/// ENGINE can not be specified for table functions.
|
||||
if (storage || !table_function_p.parse(pos, as_table_function, expected))
|
||||
{
|
||||
as_database = as_table;
|
||||
/// AS [db.]table
|
||||
if (!name_p.parse(pos, as_table, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Optional - ENGINE can be specified.
|
||||
if (!storage)
|
||||
storage_p.parse(pos, storage, expected);
|
||||
if (s_dot.ignore(pos, expected))
|
||||
{
|
||||
as_database = as_table;
|
||||
if (!name_p.parse(pos, as_table, expected))
|
||||
return false;
|
||||
}
|
||||
|
||||
/// Optional - ENGINE can be specified.
|
||||
if (!storage)
|
||||
storage_p.parse(pos, storage, expected);
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (!storage)
|
||||
{
|
||||
return false;
|
||||
}
|
||||
}
|
||||
auto comment = parseComment(pos, expected);
|
||||
|
||||
|
@ -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
|
||||
{
|
||||
|
160
src/Processors/Formats/ISchemaReader.cpp
Normal file
160
src/Processors/Formats/ISchemaReader.cpp
Normal 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;
|
||||
}
|
||||
|
||||
}
|
87
src/Processors/Formats/ISchemaReader.h
Normal file
87
src/Processors/Formats/ISchemaReader.h
Normal 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;
|
||||
};
|
||||
|
||||
}
|
@ -85,31 +85,38 @@ void ArrowBlockInputFormat::resetParser()
|
||||
record_batch_current = 0;
|
||||
}
|
||||
|
||||
static std::shared_ptr<arrow::RecordBatchReader> createStreamReader(ReadBuffer & 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());
|
||||
return *stream_reader_status;
|
||||
}
|
||||
|
||||
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);
|
||||
if (is_stopped)
|
||||
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());
|
||||
return *file_reader_status;
|
||||
}
|
||||
|
||||
|
||||
void ArrowBlockInputFormat::prepareReader()
|
||||
{
|
||||
std::shared_ptr<arrow::Schema> schema;
|
||||
|
||||
if (stream)
|
||||
{
|
||||
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();
|
||||
}
|
||||
stream_reader = createStreamReader(*in);
|
||||
else
|
||||
{
|
||||
auto arrow_file = asArrowFile(*in, format_settings, is_stopped);
|
||||
if (is_stopped)
|
||||
file_reader = createFileReader(*in, format_settings, is_stopped);
|
||||
if (!file_reader)
|
||||
return;
|
||||
|
||||
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();
|
||||
}
|
||||
|
||||
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
|
||||
|
@ -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
|
||||
|
@ -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,13 +359,15 @@ static ColumnWithTypeAndName readColumnFromArrowColumn(
|
||||
case arrow::Type::UINT16:
|
||||
{
|
||||
auto column = readColumnWithNumericData<UInt16>(arrow_column, column_name);
|
||||
column.type = std::make_shared<DataTypeDate>();
|
||||
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);
|
||||
column.type = std::make_shared<DataTypeDateTime>();
|
||||
if (read_ints_as_dates)
|
||||
column.type = std::make_shared<DataTypeDateTime>();
|
||||
return column;
|
||||
}
|
||||
case arrow::Type::TIMESTAMP:
|
||||
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
readCSVString(field, *in, format_settings.csv);
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
@ -42,17 +49,34 @@ private:
|
||||
void skipField(size_t /*file_column*/) override { skipField(); }
|
||||
void skipField();
|
||||
|
||||
void skipHeaderRow() ;
|
||||
void skipHeaderRow();
|
||||
void skipNames() override { skipHeaderRow(); }
|
||||
void skipTypes() override { skipHeaderRow(); }
|
||||
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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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();
|
||||
do
|
||||
|
||||
if (columns == 0)
|
||||
{
|
||||
if (!values.empty())
|
||||
skipFieldDelimiter();
|
||||
skipSpaces();
|
||||
values.push_back(readStringByEscapingRule(*buf, escaping_rule, format_settings));
|
||||
do
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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>();
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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>()}};
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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>();
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -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>()}};
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
|
||||
}
|
||||
|
@ -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,7 +135,20 @@ void ORCBlockInputFormat::prepareReader()
|
||||
}
|
||||
}
|
||||
|
||||
void registerInputFormatORC(FormatFactory &factory)
|
||||
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(
|
||||
"ORC",
|
||||
@ -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
|
||||
|
@ -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
|
||||
|
@ -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,7 +141,21 @@ void ParquetBlockInputFormat::prepareReader()
|
||||
}
|
||||
}
|
||||
|
||||
void registerInputFormatParquet(FormatFactory &factory)
|
||||
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(
|
||||
"Parquet",
|
||||
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -51,5 +51,14 @@ void registerInputFormatRawBLOB(FormatFactory & factory)
|
||||
});
|
||||
}
|
||||
|
||||
void registerRawBLOBSchemaReader(FormatFactory & factory)
|
||||
{
|
||||
factory.registerExternalSchemaReader("RawBLOB", [](
|
||||
const FormatSettings &)
|
||||
{
|
||||
return std::make_shared<RawBLOBSchemaReader>();
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -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>()}};
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
readEscapedString(field, *in);
|
||||
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;
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
|
||||
@ -12,13 +11,19 @@ namespace DB
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||
extern const int CANNOT_PARSE_QUOTED_STRING;
|
||||
extern const int SYNTAX_ERROR;
|
||||
extern const int ATTEMPT_TO_READ_AFTER_EOF;
|
||||
extern const int CANNOT_READ_ALL_DATA;
|
||||
extern const int CANNOT_PARSE_ESCAPE_SEQUENCE;
|
||||
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,6 +270,268 @@ void TemplateRowInputFormat::setReadBuffer(ReadBuffer & in_)
|
||||
IInputFormat::setReadBuffer(*buf);
|
||||
}
|
||||
|
||||
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_)
|
||||
{
|
||||
/// 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
|
||||
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())
|
||||
{
|
||||
/// Default format string: "${data}"
|
||||
resultset_format.delimiters.resize(2);
|
||||
resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None);
|
||||
resultset_format.format_idx_to_column_idx.emplace_back(0);
|
||||
resultset_format.column_names.emplace_back("data");
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Read format string from file
|
||||
resultset_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false,
|
||||
settings.schema.is_server, settings.schema.format_schema_path),
|
||||
[&](const String & partName) -> std::optional<size_t>
|
||||
{
|
||||
if (partName == "data")
|
||||
return 0;
|
||||
throw Exception("Unknown input part " + partName,
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
});
|
||||
}
|
||||
return resultset_format;
|
||||
}
|
||||
|
||||
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})
|
||||
@ -417,39 +542,34 @@ void registerInputFormatTemplate(FormatFactory & factory)
|
||||
IRowInputFormat::Params params,
|
||||
const FormatSettings & settings)
|
||||
{
|
||||
ParsedTemplateFormatString resultset_format;
|
||||
if (settings.template_settings.resultset_format.empty())
|
||||
auto idx_getter = [&](const String & colName) -> std::optional<size_t>
|
||||
{
|
||||
/// Default format string: "${data}"
|
||||
resultset_format.delimiters.resize(2);
|
||||
resultset_format.escaping_rules.emplace_back(ParsedTemplateFormatString::EscapingRule::None);
|
||||
resultset_format.format_idx_to_column_idx.emplace_back(0);
|
||||
resultset_format.column_names.emplace_back("data");
|
||||
}
|
||||
else
|
||||
{
|
||||
/// Read format string from file
|
||||
resultset_format = ParsedTemplateFormatString(
|
||||
FormatSchemaInfo(settings.template_settings.resultset_format, "Template", false,
|
||||
settings.schema.is_server, settings.schema.format_schema_path),
|
||||
[&](const String & partName) -> std::optional<size_t>
|
||||
{
|
||||
if (partName == "data")
|
||||
return 0;
|
||||
throw Exception("Unknown input part " + partName,
|
||||
ErrorCodes::SYNTAX_ERROR);
|
||||
});
|
||||
}
|
||||
return sample.getPositionByName(colName);
|
||||
};
|
||||
|
||||
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>
|
||||
{
|
||||
return sample.getPositionByName(colName);
|
||||
});
|
||||
return std::make_shared<TemplateRowInputFormat>(
|
||||
sample,
|
||||
buf,
|
||||
params,
|
||||
settings,
|
||||
ignore_spaces,
|
||||
fillResultSetFormat(settings),
|
||||
fillRowFormat(settings, idx_getter, true),
|
||||
settings.template_settings.row_between_delimiter);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
||||
return std::make_shared<TemplateRowInputFormat>(sample, buf, params, settings, ignore_spaces, resultset_format, row_format, 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);
|
||||
});
|
||||
}
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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);
|
||||
});
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
storage_metadata.setColumns(columns_);
|
||||
|
||||
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,
|
||||
});
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -224,7 +224,6 @@ MergeTreeData::MergeTreeData(
|
||||
{
|
||||
try
|
||||
{
|
||||
|
||||
checkPartitionKeyAndInitMinMax(metadata_.partition_key);
|
||||
setProperties(metadata_, metadata_, attach);
|
||||
if (minmax_idx_date_column_pos == -1)
|
||||
|
@ -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);
|
||||
|
@ -126,7 +126,13 @@ StorageBuffer::StorageBuffer(
|
||||
, bg_pool(getContext()->getBufferFlushSchedulePool())
|
||||
{
|
||||
StorageInMemoryMetadata storage_metadata;
|
||||
storage_metadata.setColumns(columns_);
|
||||
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);
|
||||
setInMemoryMetadata(storage_metadata);
|
||||
@ -1167,6 +1173,7 @@ void registerStorageBuffer(StorageFactory & factory)
|
||||
},
|
||||
{
|
||||
.supports_parallel_insert = true,
|
||||
.supports_schema_inference = true,
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
storage_metadata.setColumns(columns_);
|
||||
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,
|
||||
});
|
||||
}
|
||||
|
@ -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;
|
||||
};
|
||||
|
@ -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(
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -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;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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};
|
||||
|
||||
|
@ -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;
|
||||
storage_metadata.setColumns(columns_);
|
||||
|
||||
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)
|
||||
{
|
||||
|
@ -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);
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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;
|
||||
storage_metadata.setColumns(columns_);
|
||||
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,
|
||||
});
|
||||
}
|
||||
|
@ -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_,
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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")
|
||||
return;
|
||||
throw Exception("Table function '" + getName() + "' allows 2 arguments only for Distributed format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
checkIfFormatSupportsAutoStructure(getName(), format);
|
||||
return;
|
||||
}
|
||||
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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"; }
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -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
Loading…
Reference in New Issue
Block a user