Improve performance and memory usage for select of subset of columns for some formats

This commit is contained in:
avogar 2022-05-13 13:51:28 +00:00
parent e7296a2b28
commit b17fec659a
37 changed files with 145 additions and 71 deletions

View File

@ -631,7 +631,7 @@ static constexpr UInt64 operator""_GiB(unsigned long long value)
M(Bool, output_format_csv_crlf_end_of_line, false, "If it is set true, end of line in CSV format will be \\r\\n instead of \\n.", 0) \
M(Bool, input_format_csv_enum_as_number, false, "Treat inserted enum values in CSV formats as enum indices \\N", 0) \
M(Bool, input_format_csv_arrays_as_nested_csv, false, R"(When reading Array from CSV, expect that its elements were serialized in nested CSV and then put into string. Example: "[""Hello"", ""world"", ""42"""" TV""]". Braces around array can be omitted.)", 0) \
M(Bool, input_format_skip_unknown_fields, false, "Skip columns with unknown names from input data (it works for JSONEachRow, -WithNames, -WithNamesAndTypes and TSKV formats).", 0) \
M(Bool, input_format_skip_unknown_fields, true, "Skip columns with unknown names from input data (it works for JSONEachRow, -WithNames, -WithNamesAndTypes and TSKV formats).", 0) \
M(Bool, input_format_with_names_use_header, true, "For -WithNames input formats this controls whether format parser is to assume that column data appear in the input exactly as they are specified in the header.", 0) \
M(Bool, input_format_with_types_use_header, true, "For -WithNamesAndTypes input formats this controls whether format parser should check if data types from the input match data types from the header.", 0) \
M(Bool, input_format_import_nested_json, false, "Map nested JSON data to nested tables (it works for JSONEachRow format).", 0) \

View File

@ -71,7 +71,7 @@ String escapingRuleToString(FormatSettings::EscapingRule escaping_rule)
void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escaping_rule, const FormatSettings & format_settings)
{
String tmp;
NullOutput out;
constexpr const char * field_name = "<SKIPPED COLUMN>";
constexpr size_t field_name_len = 16;
switch (escaping_rule)
@ -80,19 +80,19 @@ void skipFieldByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule esca
/// Empty field, just skip spaces
break;
case FormatSettings::EscapingRule::Escaped:
readEscapedString(tmp, buf);
readEscapedStringInto(out, buf);
break;
case FormatSettings::EscapingRule::Quoted:
readQuotedFieldIntoString(tmp, buf);
readQuotedFieldInto(out, buf);
break;
case FormatSettings::EscapingRule::CSV:
readCSVString(tmp, buf, format_settings.csv);
readCSVStringInto(out, buf, format_settings.csv);
break;
case FormatSettings::EscapingRule::JSON:
skipJSONField(buf, StringRef(field_name, field_name_len));
break;
case FormatSettings::EscapingRule::Raw:
readString(tmp, buf);
readStringInto(out, buf);
break;
default:
__builtin_unreachable();
@ -219,7 +219,7 @@ String readByEscapingRule(ReadBuffer & buf, FormatSettings::EscapingRule escapin
if constexpr (read_string)
readQuotedString(result, buf);
else
readQuotedFieldIntoString(result, buf);
readQuotedField(result, buf);
break;
case FormatSettings::EscapingRule::JSON:
if constexpr (read_string)

View File

@ -538,19 +538,19 @@ void FormatFactory::markOutputFormatSupportsParallelFormatting(const String & na
}
void FormatFactory::markFormatAsColumnOriented(const String & name)
void FormatFactory::markFormatSupportsSamplingColumns(const String & name)
{
auto & target = dict[name].is_column_oriented;
auto & target = dict[name].supports_sampling_columns;
if (target)
throw Exception("FormatFactory: Format " + name + " is already marked as column oriented", ErrorCodes::LOGICAL_ERROR);
target = true;
}
bool FormatFactory::checkIfFormatIsColumnOriented(const String & name)
bool FormatFactory::checkIfFormatSupportsSamplingColumns(const String & name)
{
const auto & target = getCreators(name);
return target.is_column_oriented;
return target.supports_sampling_columns;
}
bool FormatFactory::isInputFormat(const String & name) const

View File

@ -108,7 +108,7 @@ private:
SchemaReaderCreator schema_reader_creator;
ExternalSchemaReaderCreator external_schema_reader_creator;
bool supports_parallel_formatting{false};
bool is_column_oriented{false};
bool supports_sampling_columns{false};
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
AppendSupportChecker append_support_checker;
};
@ -194,9 +194,9 @@ public:
void registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator);
void markOutputFormatSupportsParallelFormatting(const String & name);
void markFormatAsColumnOriented(const String & name);
void markFormatSupportsSamplingColumns(const String & name);
bool checkIfFormatIsColumnOriented(const String & name);
bool checkIfFormatSupportsSamplingColumns(const String & name);
bool checkIfFormatHasSchemaReader(const String & name);
bool checkIfFormatHasExternalSchemaReader(const String & name);

View File

@ -10,4 +10,10 @@ void registerWithNamesAndTypes(const std::string & base_format_name, RegisterWit
register_func(base_format_name + "WithNamesAndTypes", true, true);
}
void markFormatWithNamesAndTypesSupportsSamplingColumns(const std::string & base_format_name, FormatFactory & factory)
{
factory.markFormatSupportsSamplingColumns(base_format_name + "WithNames");
factory.markFormatSupportsSamplingColumns(base_format_name + "WithNamesAndTypes");
}
}

View File

@ -2,6 +2,7 @@
#include <string>
#include <functional>
#include <Formats/FormatFactory.h>
namespace DB
{
@ -9,4 +10,6 @@ namespace DB
using RegisterWithNamesAndTypesFunc = std::function<void(const std::string & format_name, bool with_names, bool with_types)>;
void registerWithNamesAndTypes(const std::string & base_format_name, RegisterWithNamesAndTypesFunc register_func);
void markFormatWithNamesAndTypesSupportsSamplingColumns(const std::string & base_format_name, FormatFactory & factory);
}

View File

@ -700,16 +700,18 @@ void readCSVStringInto(Vector & s, ReadBuffer & buf, const FormatSettings::CSV &
if (!buf.hasPendingData())
continue;
if constexpr (!std::is_same_v<Vector, NullOutput>)
{
/** CSV format can contain insignificant spaces and tabs.
* Usually the task of skipping them is for the calling code.
* But in this case, it will be difficult to do this, so remove the trailing whitespace by ourself.
*/
size_t size = s.size();
while (size > 0
&& (s[size - 1] == ' ' || s[size - 1] == '\t'))
while (size > 0 && (s[size - 1] == ' ' || s[size - 1] == '\t'))
--size;
s.resize(size);
}
return;
}
}
@ -741,6 +743,7 @@ void readCSVField(String & s, ReadBuffer & buf, const FormatSettings::CSV & sett
}
template void readCSVStringInto<PaddedPODArray<UInt8>>(PaddedPODArray<UInt8> & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
template void readCSVStringInto<NullOutput>(NullOutput & s, ReadBuffer & buf, const FormatSettings::CSV & settings);
template <typename Vector, typename ReturnType>
@ -1313,8 +1316,8 @@ 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)
template <typename Vector, typename ParseFunc>
static void readParsedValueInto(Vector & s, ReadBuffer & buf, ParseFunc parse_func)
{
PeekableReadBuffer peekable_buf(buf);
peekable_buf.setCheckpoint();
@ -1326,8 +1329,8 @@ static void readParsedValueIntoString(String & s, ReadBuffer & buf, ParseFunc pa
peekable_buf.position() = end;
}
template <char opening_bracket, char closing_bracket>
static void readQuotedFieldInBrackets(String & s, ReadBuffer & buf)
template <char opening_bracket, char closing_bracket, typename Vector>
static void readQuotedFieldInBracketsInto(Vector & s, ReadBuffer & buf)
{
assertChar(opening_bracket, buf);
s.push_back(opening_bracket);
@ -1363,10 +1366,9 @@ static void readQuotedFieldInBrackets(String & s, ReadBuffer & buf)
}
}
void readQuotedFieldIntoString(String & s, ReadBuffer & buf)
template <typename Vector>
void readQuotedFieldInto(Vector & s, ReadBuffer & buf)
{
s.clear();
if (buf.eof())
return;
@ -1386,11 +1388,11 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf)
s.push_back('\'');
}
else if (*buf.position() == '[')
readQuotedFieldInBrackets<'[', ']'>(s, buf);
readQuotedFieldInBracketsInto<'[', ']'>(s, buf);
else if (*buf.position() == '(')
readQuotedFieldInBrackets<'(', ')'>(s, buf);
readQuotedFieldInBracketsInto<'(', ')'>(s, buf);
else if (*buf.position() == '{')
readQuotedFieldInBrackets<'{', '}'>(s, buf);
readQuotedFieldInBracketsInto<'{', '}'>(s, buf);
else if (checkCharCaseInsensitive('n', buf))
{
/// NULL or NaN
@ -1423,14 +1425,20 @@ void readQuotedFieldIntoString(String & s, ReadBuffer & buf)
Float64 tmp;
readFloatText(tmp, in);
};
readParsedValueIntoString(s, buf, parse_func);
readParsedValueInto(s, buf, parse_func);
}
}
void readQuotedField(String & s, ReadBuffer & buf)
{
s.clear();
readQuotedFieldInto(s, buf);
}
void readJSONFieldIntoString(String & s, ReadBuffer & buf)
{
auto parse_func = [](ReadBuffer & in) { skipJSONField(in, "json_field"); };
readParsedValueIntoString(s, buf, parse_func);
readParsedValueInto(s, buf, parse_func);
}
}

View File

@ -1425,7 +1425,10 @@ struct PcgDeserializer
}
};
void readQuotedFieldIntoString(String & s, ReadBuffer & buf);
template <typename Vector>
void readQuotedFieldInto(Vector & s, ReadBuffer & buf);
void readQuotedField(String & s, ReadBuffer & buf);
void readJSONFieldIntoString(String & s, ReadBuffer & buf);

View File

@ -188,7 +188,7 @@ void registerInputFormatArrow(FormatFactory & factory)
{
return std::make_shared<ArrowBlockInputFormat>(buf, sample, false, format_settings);
});
factory.markFormatAsColumnOriented("Arrow");
factory.markFormatSupportsSamplingColumns("Arrow");
factory.registerInputFormat(
"ArrowStream",
[](ReadBuffer & buf,

View File

@ -114,6 +114,7 @@ void registerInputFormatRowBinary(FormatFactory & factory)
};
registerWithNamesAndTypes("RowBinary", register_func);
factory.markFormatSupportsSamplingColumns("RowBinaryWithNamesAndTypes");
factory.registerFileExtension("bin", "RowBinary");
}

View File

@ -112,7 +112,9 @@ String CSVFormatReader::readCSVFieldIntoString()
void CSVFormatReader::skipField()
{
readCSVFieldIntoString<true>();
skipWhitespacesAndTabs(*in);
NullOutput out;
readCSVStringInto(out, *in, format_settings.csv);
}
void CSVFormatReader::skipRowEndDelimiter()
@ -374,6 +376,7 @@ void registerFileSegmentationEngineCSV(FormatFactory & factory)
};
registerWithNamesAndTypes("CSV", register_func);
markFormatWithNamesAndTypesSupportsSamplingColumns("CSV", factory);
}
void registerCSVSchemaReader(FormatFactory & factory)

View File

@ -310,6 +310,7 @@ void registerInputFormatCapnProto(FormatFactory & factory)
return std::make_shared<CapnProtoRowInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "CapnProto", true), settings);
});
factory.markFormatSupportsSamplingColumns("CapnProto");
factory.registerFileExtension("capnp", "CapnProto");
}

View File

@ -333,6 +333,7 @@ void registerInputFormatCustomSeparated(FormatFactory & factory)
});
};
registerWithNamesAndTypes(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", register_func);
markFormatWithNamesAndTypesSupportsSamplingColumns(ignore_spaces ? "CustomSeparatedIgnoreSpaces" : "CustomSeparated", factory);
}
}

View File

@ -229,6 +229,7 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
};
registerWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
markFormatWithNamesAndTypesSupportsSamplingColumns(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", factory);
}
}

View File

@ -393,6 +393,11 @@ void registerInputFormatJSONEachRow(FormatFactory & factory)
{
return std::make_shared<JSONEachRowRowInputFormat>(buf, sample, std::move(params), settings, true);
});
factory.markFormatSupportsSamplingColumns("JSONEachRow");
factory.markFormatSupportsSamplingColumns("JSONLines");
factory.markFormatSupportsSamplingColumns("NDJSON");
factory.markFormatSupportsSamplingColumns("JSONStringsEachRow");
}
void registerFileSegmentationEngineJSONEachRow(FormatFactory & factory)

View File

@ -397,8 +397,8 @@ bool MySQLDumpRowInputFormat::readField(IColumn & column, size_t column_idx)
void MySQLDumpRowInputFormat::skipField()
{
String tmp;
readQuotedFieldIntoString(tmp, *in);
NullOutput out;
readQuotedFieldInto(out, *in);
}
MySQLDumpSchemaReader::MySQLDumpSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
@ -434,7 +434,7 @@ DataTypes MySQLDumpSchemaReader::readRowAndGetDataTypes()
if (!data_types.empty())
skipFieldDelimiter(in);
readQuotedFieldIntoString(value, in);
readQuotedField(value, in);
auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
data_types.push_back(std::move(type));
}

View File

@ -200,7 +200,7 @@ void registerInputFormatORC(FormatFactory & factory)
{
return std::make_shared<ORCBlockInputFormat>(buf, sample, settings);
});
factory.markFormatAsColumnOriented("ORC");
factory.markFormatSupportsSamplingColumns("ORC");
}
void registerORCSchemaReader(FormatFactory & factory)

View File

@ -193,7 +193,7 @@ void registerInputFormatParquet(FormatFactory & factory)
{
return std::make_shared<ParquetBlockInputFormat>(buf, sample, settings);
});
factory.markFormatAsColumnOriented("Parquet");
factory.markFormatSupportsSamplingColumns("Parquet");
}
void registerParquetSchemaReader(FormatFactory & factory)

View File

@ -79,7 +79,7 @@ void registerInputFormatProtobufList(FormatFactory & factory)
return std::make_shared<ProtobufListInputFormat>(buf, sample, std::move(params),
FormatSchemaInfo(settings, "Protobuf", true), settings.protobuf.input_flatten_google_wrappers);
});
factory.markFormatAsColumnOriented("ProtobufList");
factory.markFormatSupportsSamplingColumns("ProtobufList");
}
void registerProtobufListSchemaReader(FormatFactory & factory)

View File

@ -69,6 +69,7 @@ void registerInputFormatProtobuf(FormatFactory & factory)
with_length_delimiter,
settings.protobuf.input_flatten_google_wrappers);
});
factory.markFormatSupportsSamplingColumns(with_length_delimiter ? "Protobuf" : "ProtobufSingle");
}
}

View File

@ -280,6 +280,8 @@ void registerInputFormatTSKV(FormatFactory & factory)
{
return std::make_shared<TSKVRowInputFormat>(buf, sample, std::move(params), settings);
});
factory.markFormatSupportsSamplingColumns("TSKV");
}
void registerTSKVSchemaReader(FormatFactory & factory)
{

View File

@ -80,7 +80,11 @@ String TabSeparatedFormatReader::readFieldIntoString()
void TabSeparatedFormatReader::skipField()
{
readFieldIntoString();
NullOutput out;
if (is_raw)
readStringInto(out, *in);
else
readEscapedStringInto(out, *in);
}
void TabSeparatedFormatReader::skipHeaderRow()
@ -347,6 +351,8 @@ void registerFileSegmentationEngineTabSeparated(FormatFactory & factory)
registerWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
registerWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
markFormatWithNamesAndTypesSupportsSamplingColumns(is_raw ? "TSVRaw" : "TSV", factory);
markFormatWithNamesAndTypesSupportsSamplingColumns(is_raw ? "TabSeparatedRaw" : "TabSeparated", factory);
}
// We can use the same segmentation engine for TSKV.

View File

@ -599,7 +599,7 @@ DataTypes ValuesSchemaReader::readRowAndGetDataTypes()
skipWhitespaceIfAny(buf);
}
readQuotedFieldIntoString(value, buf);
readQuotedField(value, buf);
auto type = determineDataTypeByEscapingRule(value, format_settings, FormatSettings::EscapingRule::Quoted);
data_types.push_back(std::move(type));
}

View File

@ -476,9 +476,9 @@ private:
};
bool StorageHDFS::isColumnOriented() const
bool StorageHDFS::supportsSamplingColumns() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSamplingColumns(format_name);
}
Pipe StorageHDFS::read(
@ -527,7 +527,7 @@ Pipe StorageHDFS::read(
ColumnsDescription columns_description;
Block block_for_format;
if (isColumnOriented())
if (supportsSamplingColumns())
{
auto fetch_columns = column_names;
const auto & virtuals = getVirtuals();

View File

@ -57,7 +57,7 @@ public:
/// Is is useful because column oriented formats could effectively skip unknown columns
/// So we can create a header of only required columns in read method and ask
/// format to read only them. Note: this hack cannot be done with ordinary formats like TSV.
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
static ColumnsDescription getTableStructureFromData(
const String & format,

View File

@ -622,14 +622,14 @@ HiveFilePtr StorageHive::getHiveFileIfNeeded(
return hive_file;
}
bool StorageHive::isColumnOriented() const
bool StorageHive::supportsSamplingColumns() const
{
return format_name == "Parquet" || format_name == "ORC";
}
void StorageHive::getActualColumnsToRead(Block & sample_block, const Block & header_block, const NameSet & partition_columns) const
{
if (!isColumnOriented())
if (!supportsSamplingColumns())
sample_block = header_block;
UInt32 erased_columns = 0;
for (const auto & column : partition_columns)
@ -795,7 +795,7 @@ std::optional<UInt64>
StorageHive::totalRowsImpl(const Settings & settings, const SelectQueryInfo & query_info, ContextPtr context_, PruneLevel prune_level) const
{
/// Row-based format like Text doesn't support totalRowsByPartitionPredicate
if (!isColumnOriented())
if (!supportsSamplingColumns())
return {};
auto hive_metastore_client = HiveMetastoreClientFactory::instance().getOrCreate(hive_metastore_url);

View File

@ -63,7 +63,7 @@ public:
NamesAndTypesList getVirtuals() const override;
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
std::optional<UInt64> totalRows(const Settings & settings) const override;
std::optional<UInt64> totalRowsByPartitionPredicate(const SelectQueryInfo & query_info, ContextPtr context_) const override;

View File

@ -585,7 +585,7 @@ public:
/// Returns true if all disks of storage are read-only.
virtual bool isStaticStorage() const;
virtual bool isColumnOriented() const { return false; }
virtual bool supportsSamplingColumns() const { return false; }
/// If it is possible to quickly determine exact number of rows in the table at this moment of time, then return it.
/// Used for:

View File

@ -316,9 +316,9 @@ ColumnsDescription StorageFile::getTableStructureFromFile(
return readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context);
}
bool StorageFile::isColumnOriented() const
bool StorageFile::supportsSamplingColumns() const
{
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
return format_name != "Distributed" && FormatFactory::instance().checkIfFormatSupportsSamplingColumns(format_name);
}
StorageFile::StorageFile(int table_fd_, CommonArguments args)
@ -465,7 +465,7 @@ public:
const ColumnsDescription & columns_description,
const FilesInfoPtr & files_info)
{
if (storage->isColumnOriented())
if (storage->supportsSamplingColumns())
return storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
else
return getHeader(storage_snapshot->metadata, files_info->need_path_column, files_info->need_file_column);
@ -530,7 +530,7 @@ public:
auto get_block_for_format = [&]() -> Block
{
if (storage->isColumnOriented())
if (storage->supportsSamplingColumns())
return storage_snapshot->getSampleBlockForColumns(columns_description.getNamesOfPhysical());
return storage_snapshot->metadata->getSampleBlock();
};
@ -690,7 +690,7 @@ Pipe StorageFile::read(
{
const auto get_columns_for_format = [&]() -> ColumnsDescription
{
if (isColumnOriented())
if (supportsSamplingColumns())
return ColumnsDescription{
storage_snapshot->getSampleBlockForColumns(column_names).getNamesAndTypesList()};
else

View File

@ -69,11 +69,11 @@ public:
static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read);
/// Check if the format is column-oriented.
/// Is is useful because column oriented formats could effectively skip unknown columns
/// Check if the format supports reading only some sampling of columns.
/// Is is useful because such formats could effectively skip unknown columns
/// So we can create a header of only required columns in read method and ask
/// format to read only them. Note: this hack cannot be done with ordinary formats like TSV.
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
bool supportsPartitionBy() const override { return true; }

View File

@ -676,9 +676,9 @@ std::shared_ptr<StorageS3Source::IteratorWrapper> StorageS3::createFileIterator(
}
}
bool StorageS3::isColumnOriented() const
bool StorageS3::supportsSamplingColumns() const
{
return FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
return FormatFactory::instance().checkIfFormatSupportsSamplingColumns(format_name);
}
Pipe StorageS3::read(
@ -707,7 +707,7 @@ Pipe StorageS3::read(
ColumnsDescription columns_description;
Block block_for_format;
if (isColumnOriented())
if (supportsSamplingColumns())
{
auto fetch_columns = column_names;
const auto & virtuals = getVirtuals();

View File

@ -234,7 +234,7 @@ private:
ContextPtr ctx,
std::vector<String> * read_keys_in_distributed_processing = nullptr);
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
};
}

View File

@ -582,9 +582,9 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData(
return readSchemaFromFormat(format, format_settings, read_buffer_iterator, urls_to_check.size() > 1, context);
}
bool IStorageURLBase::isColumnOriented() const
bool IStorageURLBase::supportsSamplingColumns() const
{
return FormatFactory::instance().checkIfFormatIsColumnOriented(format_name);
return FormatFactory::instance().checkIfFormatSupportsSamplingColumns(format_name);
}
Pipe IStorageURLBase::read(
@ -600,7 +600,7 @@ Pipe IStorageURLBase::read(
ColumnsDescription columns_description;
Block block_for_format;
if (isColumnOriented())
if (supportsSamplingColumns())
{
columns_description = ColumnsDescription{
storage_snapshot->getSampleBlockForColumns(column_names).getNamesAndTypesList()};
@ -688,7 +688,7 @@ Pipe StorageURLWithFailover::read(
{
ColumnsDescription columns_description;
Block block_for_format;
if (isColumnOriented())
if (supportsSamplingColumns())
{
columns_description = ColumnsDescription{
storage_snapshot->getSampleBlockForColumns(column_names).getNamesAndTypesList()};

View File

@ -93,7 +93,7 @@ protected:
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size) const;
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
private:
virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0;

View File

@ -140,7 +140,7 @@ SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMet
chooseCompressionMethod(uri, compression_method));
}
bool StorageXDBC::isColumnOriented() const
bool StorageXDBC::supportsSamplingColumns() const
{
return true;
}

View File

@ -67,7 +67,7 @@ private:
Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const override;
bool isColumnOriented() const override;
bool supportsSamplingColumns() const override;
};
}

View File

@ -0,0 +1,33 @@
<test>
<settings>
<max_threads>1</max_threads>
</settings>
<substitutions>
<substitution>
<name>format</name>
<values>
<value>TabSeparatedWithNames</value>
<value>TabSeparatedRawWithNames</value>
<value>CustomSeparatedWithNames</value>
<value>CSVWithNames</value>
<value>JSONEachRow</value>
<value>JSONCompactEachRowWithNames</value>
<value>TSKV</value>
<value>RowBinaryWithNamesAndTypes</value>
<value>Avro</value>
<value>ORC</value>
<value>Parquet</value>
<value>Arrow</value>
</values>
</substitution>
</substitutions>
<create_query>CREATE TABLE IF NOT EXISTS table_{format} ENGINE = File({format}) AS test.hits</create_query>
<fill_query>INSERT INTO table_{format} SELECT * FROM test.hits LIMIT 100000</fill_query>
<query>SELECT WatchID FROM table_{format} FORMAT Null</query>
<drop_query>DROP TABLE IF EXISTS table_{format}</drop_query>
</test>