mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Fix WithNamesAndTypes parallel parsing, add new tests, small refactoring
This commit is contained in:
parent
74fd33b62e
commit
7007286088
@ -591,6 +591,7 @@
|
|||||||
M(621, CANNOT_NORMALIZE_STRING) \
|
M(621, CANNOT_NORMALIZE_STRING) \
|
||||||
M(622, CANNOT_PARSE_CAPN_PROTO_SCHEMA) \
|
M(622, CANNOT_PARSE_CAPN_PROTO_SCHEMA) \
|
||||||
M(623, CAPN_PROTO_BAD_CAST) \
|
M(623, CAPN_PROTO_BAD_CAST) \
|
||||||
|
M(624, CANNOT_SKIP_UNKNOWN_FIELD) \
|
||||||
\
|
\
|
||||||
M(999, KEEPER_EXCEPTION) \
|
M(999, KEEPER_EXCEPTION) \
|
||||||
M(1000, POCO_EXCEPTION) \
|
M(1000, POCO_EXCEPTION) \
|
||||||
|
@ -67,8 +67,6 @@ public:
|
|||||||
const Columns & columns,
|
const Columns & columns,
|
||||||
size_t row)>;
|
size_t row)>;
|
||||||
|
|
||||||
private:
|
|
||||||
|
|
||||||
using InputCreatorFunc = InputFormatPtr(
|
using InputCreatorFunc = InputFormatPtr(
|
||||||
ReadBuffer & buf,
|
ReadBuffer & buf,
|
||||||
const Block & header,
|
const Block & header,
|
||||||
@ -83,6 +81,7 @@ private:
|
|||||||
const RowOutputFormatParams & params,
|
const RowOutputFormatParams & params,
|
||||||
const FormatSettings & settings)>;
|
const FormatSettings & settings)>;
|
||||||
|
|
||||||
|
private:
|
||||||
/// Some input formats can have non trivial readPrefix() and readSuffix(),
|
/// Some input formats can have non trivial readPrefix() and readSuffix(),
|
||||||
/// so in some cases there is no possibility to use parallel parsing.
|
/// so in some cases there is no possibility to use parallel parsing.
|
||||||
/// The checker should return true if parallel parsing should be disabled.
|
/// The checker should return true if parallel parsing should be disabled.
|
||||||
|
@ -14,7 +14,7 @@ namespace ErrorCodes
|
|||||||
}
|
}
|
||||||
|
|
||||||
template <const char opening_bracket, const char closing_bracket>
|
template <const char opening_bracket, const char closing_bracket>
|
||||||
static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows)
|
||||||
{
|
{
|
||||||
skipWhitespaceIfAny(in);
|
skipWhitespaceIfAny(in);
|
||||||
|
|
||||||
@ -23,7 +23,7 @@ static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer
|
|||||||
bool quotes = false;
|
bool quotes = false;
|
||||||
size_t number_of_rows = 0;
|
size_t number_of_rows = 0;
|
||||||
|
|
||||||
while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast<size_t>(pos - in.position()) < min_chunk_size))
|
while (loadAtPosition(in, memory, pos) && (balance || memory.size() + static_cast<size_t>(pos - in.position()) < min_chunk_size || number_of_rows < min_rows))
|
||||||
{
|
{
|
||||||
const auto current_object_size = memory.size() + static_cast<size_t>(pos - in.position());
|
const auto current_object_size = memory.size() + static_cast<size_t>(pos - in.position());
|
||||||
if (current_object_size > 10 * min_chunk_size)
|
if (current_object_size > 10 * min_chunk_size)
|
||||||
@ -94,12 +94,12 @@ static std::pair<bool, size_t> fileSegmentationEngineJSONEachRowImpl(ReadBuffer
|
|||||||
|
|
||||||
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
||||||
{
|
{
|
||||||
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size);
|
return fileSegmentationEngineJSONEachRowImpl<'{', '}'>(in, memory, min_chunk_size, 1);
|
||||||
}
|
}
|
||||||
|
|
||||||
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(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)
|
||||||
{
|
{
|
||||||
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size);
|
return fileSegmentationEngineJSONEachRowImpl<'[', ']'>(in, memory, min_chunk_size, min_rows);
|
||||||
}
|
}
|
||||||
|
|
||||||
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
|
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf)
|
||||||
|
@ -9,7 +9,7 @@ namespace DB
|
|||||||
{
|
{
|
||||||
|
|
||||||
std::pair<bool, size_t> fileSegmentationEngineJSONEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size);
|
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);
|
std::pair<bool, size_t> fileSegmentationEngineJSONCompactEachRow(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows);
|
||||||
|
|
||||||
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
|
bool nonTrivialPrefixAndSuffixCheckerJSONEachRowImpl(ReadBuffer & buf);
|
||||||
|
|
||||||
|
39
src/Formats/registerWithNamesAndTypes.cpp
Normal file
39
src/Formats/registerWithNamesAndTypes.cpp
Normal file
@ -0,0 +1,39 @@
|
|||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
void registerInputFormatWithNamesAndTypes(FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator)
|
||||||
|
{
|
||||||
|
factory.registerInputFormat(base_format_name, get_input_creator(false, false));
|
||||||
|
factory.registerInputFormat(base_format_name + "WithNames", get_input_creator(true, false));
|
||||||
|
factory.registerInputFormat(base_format_name + "WithNamesAndTypes", get_input_creator(true, true));
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerOutputFormatWithNamesAndTypes(
|
||||||
|
FormatFactory & factory,
|
||||||
|
const String & base_format_name,
|
||||||
|
GetOutputCreatorWithNamesAndTypesFunc get_output_creator,
|
||||||
|
bool supports_parallel_formatting)
|
||||||
|
{
|
||||||
|
factory.registerOutputFormat(base_format_name, get_output_creator(false, false));
|
||||||
|
factory.registerOutputFormat(base_format_name + "WithNames", get_output_creator(true, false));
|
||||||
|
factory.registerOutputFormat(base_format_name + "WithNamesAndTypes", get_output_creator(true, true));
|
||||||
|
|
||||||
|
if (supports_parallel_formatting)
|
||||||
|
{
|
||||||
|
factory.markOutputFormatSupportsParallelFormatting(base_format_name);
|
||||||
|
factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNames");
|
||||||
|
factory.markOutputFormatSupportsParallelFormatting(base_format_name + "WithNamesAndTypes");
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
||||||
|
FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine)
|
||||||
|
{
|
||||||
|
factory.registerFileSegmentationEngine(base_format_name, get_file_segmentation_engine(1));
|
||||||
|
factory.registerFileSegmentationEngine(base_format_name + "WithNames", get_file_segmentation_engine(2));
|
||||||
|
factory.registerFileSegmentationEngine(base_format_name + "WithNamesAndTypes", get_file_segmentation_engine(3));
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
23
src/Formats/registerWithNamesAndTypes.h
Normal file
23
src/Formats/registerWithNamesAndTypes.h
Normal file
@ -0,0 +1,23 @@
|
|||||||
|
#pragma once
|
||||||
|
|
||||||
|
#include <Formats/FormatFactory.h>
|
||||||
|
|
||||||
|
namespace DB
|
||||||
|
{
|
||||||
|
|
||||||
|
using GetInputCreatorWithNamesAndTypesFunc = std::function<FormatFactory::InputCreator(bool with_names, bool with_types)>;
|
||||||
|
void registerInputFormatWithNamesAndTypes(
|
||||||
|
FormatFactory & factory, const String & base_format_name, GetInputCreatorWithNamesAndTypesFunc get_input_creator);
|
||||||
|
|
||||||
|
using GetOutputCreatorWithNamesAndTypesFunc = std::function<FormatFactory::OutputCreator(bool with_names, bool with_types)>;
|
||||||
|
void registerOutputFormatWithNamesAndTypes(
|
||||||
|
FormatFactory & factory,
|
||||||
|
const String & base_format_name,
|
||||||
|
GetOutputCreatorWithNamesAndTypesFunc get_output_creator,
|
||||||
|
bool supports_parallel_formatting = false);
|
||||||
|
|
||||||
|
using GetFileSegmentationEngineWithNamesAndTypesFunc = std::function<FormatFactory::FileSegmentationEngine(size_t min_rows)>;
|
||||||
|
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
||||||
|
FormatFactory & factory, const String & base_format_name, GetFileSegmentationEngineWithNamesAndTypesFunc get_file_segmentation_engine);
|
||||||
|
|
||||||
|
}
|
@ -113,11 +113,4 @@ void IRowOutputFormat::writeTotals(const DB::Columns & columns, size_t row_num)
|
|||||||
write(columns, row_num);
|
write(columns, row_num);
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func)
|
|
||||||
{
|
|
||||||
register_func(base_format_name, false, false);
|
|
||||||
register_func(base_format_name + "WithNames", true, false);
|
|
||||||
register_func(base_format_name + "WithNamesAndTypes", true, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -87,7 +87,4 @@ private:
|
|||||||
|
|
||||||
};
|
};
|
||||||
|
|
||||||
using RegisterOutputFormatWithNamesAndTypes = std::function<void(const String & format_name, bool with_names, bool with_types)>;
|
|
||||||
void registerOutputFormatWithNamesAndTypes(const String & base_format_name, RegisterOutputFormatWithNamesAndTypes register_func);
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <IO/ReadHelpers.h>
|
#include <IO/ReadHelpers.h>
|
||||||
#include <Processors/Formats/Impl/BinaryRowInputFormat.h>
|
#include <Processors/Formats/Impl/BinaryRowInputFormat.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
#include <DataTypes/DataTypeFactory.h>
|
#include <DataTypes/DataTypeFactory.h>
|
||||||
|
|
||||||
|
|
||||||
@ -10,11 +11,11 @@ namespace DB
|
|||||||
|
|
||||||
namespace ErrorCodes
|
namespace ErrorCodes
|
||||||
{
|
{
|
||||||
extern const int LOGICAL_ERROR;
|
extern const int CANNOT_SKIP_UNKNOWN_FIELD;
|
||||||
}
|
}
|
||||||
|
|
||||||
BinaryRowInputFormat::BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_)
|
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_and_types, with_names_and_types, format_settings_)
|
: RowInputFormatWithNamesAndTypes(std::move(header), in_, std::move(params_), with_names_, with_types_, format_settings_)
|
||||||
{
|
{
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -71,30 +72,26 @@ void BinaryRowInputFormat::skipTypes()
|
|||||||
void BinaryRowInputFormat::skipField(size_t file_column)
|
void BinaryRowInputFormat::skipField(size_t file_column)
|
||||||
{
|
{
|
||||||
if (file_column >= read_data_types.size())
|
if (file_column >= read_data_types.size())
|
||||||
throw Exception(ErrorCodes::LOGICAL_ERROR, "Cannot skip field in Binary format, because it's type is unknown");
|
throw Exception(ErrorCodes::CANNOT_SKIP_UNKNOWN_FIELD, "Cannot skip unknown field in RowBinaryWithNames format, because it's type is unknown");
|
||||||
Field field;
|
Field field;
|
||||||
read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in);
|
read_data_types[file_column]->getDefaultSerialization()->deserializeBinary(field, *in);
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerInputFormatRowBinary(FormatFactory & factory)
|
void registerInputFormatRowBinary(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerInputFormat("RowBinary", [](
|
auto get_input_creator = [](bool with_names, bool with_types)
|
||||||
|
{
|
||||||
|
return [with_names, with_types](
|
||||||
ReadBuffer & buf,
|
ReadBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
const IRowInputFormat::Params & params,
|
const IRowInputFormat::Params & params,
|
||||||
const FormatSettings & settings)
|
const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<BinaryRowInputFormat>(buf, sample, params, false, settings);
|
return std::make_shared<BinaryRowInputFormat>(buf, sample, params, with_names, with_types, settings);
|
||||||
});
|
};
|
||||||
|
};
|
||||||
|
|
||||||
factory.registerInputFormat("RowBinaryWithNamesAndTypes", [](
|
registerInputFormatWithNamesAndTypes(factory, "RowBinary", get_input_creator);
|
||||||
ReadBuffer & buf,
|
|
||||||
const Block & sample,
|
|
||||||
const IRowInputFormat::Params & params,
|
|
||||||
const FormatSettings & settings)
|
|
||||||
{
|
|
||||||
return std::make_shared<BinaryRowInputFormat>(buf, sample, params, true, settings);
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -16,7 +16,7 @@ class ReadBuffer;
|
|||||||
class BinaryRowInputFormat : public RowInputFormatWithNamesAndTypes
|
class BinaryRowInputFormat : public RowInputFormatWithNamesAndTypes
|
||||||
{
|
{
|
||||||
public:
|
public:
|
||||||
BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_and_types, const FormatSettings & format_settings_);
|
BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
|
||||||
|
|
||||||
String getName() const override { return "BinaryRowInputFormat"; }
|
String getName() const override { return "BinaryRowInputFormat"; }
|
||||||
|
|
||||||
|
@ -4,6 +4,7 @@
|
|||||||
#include <DataTypes/IDataType.h>
|
#include <DataTypes/IDataType.h>
|
||||||
#include <Processors/Formats/Impl/BinaryRowOutputFormat.h>
|
#include <Processors/Formats/Impl/BinaryRowOutputFormat.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -49,23 +50,19 @@ void BinaryRowOutputFormat::writeField(const IColumn & column, const ISerializat
|
|||||||
|
|
||||||
void registerOutputFormatRowBinary(FormatFactory & factory)
|
void registerOutputFormatRowBinary(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
factory.registerOutputFormat("RowBinary", [](
|
auto get_output_creator = [&](bool with_names, bool with_types)
|
||||||
|
{
|
||||||
|
return [with_names, with_types](
|
||||||
WriteBuffer & buf,
|
WriteBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
const RowOutputFormatParams & params,
|
const RowOutputFormatParams & params,
|
||||||
const FormatSettings &)
|
const FormatSettings &)
|
||||||
{
|
{
|
||||||
return std::make_shared<BinaryRowOutputFormat>(buf, sample, false, false, params);
|
return std::make_shared<BinaryRowOutputFormat>(buf, sample, with_names, with_types, params);
|
||||||
});
|
};
|
||||||
|
};
|
||||||
|
|
||||||
factory.registerOutputFormat("RowBinaryWithNamesAndTypes", [](
|
registerOutputFormatWithNamesAndTypes(factory, "RowBinary", get_output_creator);
|
||||||
WriteBuffer & buf,
|
|
||||||
const Block & sample,
|
|
||||||
const RowOutputFormatParams & params,
|
|
||||||
const FormatSettings &)
|
|
||||||
{
|
|
||||||
return std::make_shared<BinaryRowOutputFormat>(buf, sample, true, true, params);
|
|
||||||
});
|
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -3,8 +3,9 @@
|
|||||||
#include <IO/Operators.h>
|
#include <IO/Operators.h>
|
||||||
|
|
||||||
#include <Formats/verbosePrintString.h>
|
#include <Formats/verbosePrintString.h>
|
||||||
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Processors/Formats/Impl/CSVRowInputFormat.h>
|
||||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||||
#include <DataTypes/DataTypeNothing.h>
|
#include <DataTypes/DataTypeNothing.h>
|
||||||
|
|
||||||
@ -232,22 +233,22 @@ bool CSVRowInputFormat::readField(IColumn & column, const DataTypePtr & type, co
|
|||||||
|
|
||||||
void registerInputFormatCSV(FormatFactory & factory)
|
void registerInputFormatCSV(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_input_creator = [](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerInputFormat(format_name, [with_names, with_types](
|
return [with_names, with_types](
|
||||||
ReadBuffer & buf,
|
ReadBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
IRowInputFormat::Params params,
|
IRowInputFormat::Params params,
|
||||||
const FormatSettings & settings)
|
const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<CSVRowInputFormat>(sample, buf, std::move(params), with_names, with_types, settings);
|
return std::make_shared<CSVRowInputFormat>(sample, buf, std::move(params), with_names, with_types, settings);
|
||||||
});
|
};
|
||||||
};
|
};
|
||||||
|
|
||||||
registerInputFormatWithNamesAndTypes("CSV", register_func);
|
registerInputFormatWithNamesAndTypes(factory, "CSV", get_input_creator);
|
||||||
}
|
}
|
||||||
|
|
||||||
static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, size_t min_rows)
|
||||||
{
|
{
|
||||||
char * pos = in.position();
|
char * pos = in.position();
|
||||||
bool quotes = false;
|
bool quotes = false;
|
||||||
@ -287,7 +288,7 @@ static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB
|
|||||||
else if (*pos == '\n')
|
else if (*pos == '\n')
|
||||||
{
|
{
|
||||||
++number_of_rows;
|
++number_of_rows;
|
||||||
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
|
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size && number_of_rows >= min_rows)
|
||||||
need_more_data = false;
|
need_more_data = false;
|
||||||
++pos;
|
++pos;
|
||||||
if (loadAtPosition(in, memory, pos) && *pos == '\r')
|
if (loadAtPosition(in, memory, pos) && *pos == '\r')
|
||||||
@ -295,7 +296,7 @@ static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB
|
|||||||
}
|
}
|
||||||
else if (*pos == '\r')
|
else if (*pos == '\r')
|
||||||
{
|
{
|
||||||
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
|
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size && number_of_rows >= min_rows)
|
||||||
need_more_data = false;
|
need_more_data = false;
|
||||||
++pos;
|
++pos;
|
||||||
if (loadAtPosition(in, memory, pos) && *pos == '\n')
|
if (loadAtPosition(in, memory, pos) && *pos == '\n')
|
||||||
@ -313,7 +314,14 @@ static std::pair<bool, size_t> fileSegmentationEngineCSVImpl(ReadBuffer & in, DB
|
|||||||
|
|
||||||
void registerFileSegmentationEngineCSV(FormatFactory & factory)
|
void registerFileSegmentationEngineCSV(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", &fileSegmentationEngineCSVImpl);
|
auto get_file_segmentation_engine = [](size_t min_rows)
|
||||||
|
{
|
||||||
|
return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
||||||
|
{
|
||||||
|
return fileSegmentationEngineCSVImpl(in, memory, min_chunk_size, min_rows);
|
||||||
|
};
|
||||||
|
};
|
||||||
|
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "CSV", get_file_segmentation_engine);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Processors/Formats/Impl/CSVRowOutputFormat.h>
|
#include <Processors/Formats/Impl/CSVRowOutputFormat.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
|
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
@ -74,20 +75,19 @@ void CSVRowOutputFormat::writeBeforeExtremes()
|
|||||||
|
|
||||||
void registerOutputFormatCSV(FormatFactory & factory)
|
void registerOutputFormatCSV(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_output_creator = [](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerOutputFormat(format_name, [=](
|
return [with_names, with_types](
|
||||||
WriteBuffer & buf,
|
WriteBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
const RowOutputFormatParams & params,
|
const RowOutputFormatParams & params,
|
||||||
const FormatSettings & format_settings)
|
const FormatSettings & format_settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<CSVRowOutputFormat>(buf, sample, with_names, with_types, params, format_settings);
|
return std::make_shared<CSVRowOutputFormat>(buf, sample, with_names, with_types, params, format_settings);
|
||||||
});
|
};
|
||||||
factory.markOutputFormatSupportsParallelFormatting(format_name);
|
|
||||||
};
|
};
|
||||||
|
|
||||||
registerOutputFormatWithNamesAndTypes("CSV", register_func);
|
registerOutputFormatWithNamesAndTypes(factory, "CSV", get_output_creator, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
#include <Formats/verbosePrintString.h>
|
#include <Formats/verbosePrintString.h>
|
||||||
#include <Formats/JSONEachRowUtils.h>
|
#include <Formats/JSONEachRowUtils.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
#include <DataTypes/NestedUtils.h>
|
#include <DataTypes/NestedUtils.h>
|
||||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||||
|
|
||||||
@ -183,26 +184,34 @@ void registerInputFormatJSONCompactEachRow(FormatFactory & factory)
|
|||||||
{
|
{
|
||||||
for (bool yield_strings : {true, false})
|
for (bool yield_strings : {true, false})
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_input_creator = [yield_strings](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerInputFormat(format_name, [with_names, with_types, yield_strings](
|
return [with_names, with_types, yield_strings](
|
||||||
ReadBuffer & buf,
|
ReadBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
IRowInputFormat::Params params,
|
IRowInputFormat::Params params,
|
||||||
const FormatSettings & settings)
|
const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<JSONCompactEachRowRowInputFormat>(sample, buf, std::move(params), with_names, with_types, yield_strings, settings);
|
return std::make_shared<JSONCompactEachRowRowInputFormat>(sample, buf, std::move(params), with_names, with_types, yield_strings, settings);
|
||||||
});
|
};
|
||||||
};
|
};
|
||||||
|
|
||||||
registerInputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
|
registerInputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_input_creator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory)
|
void registerFileSegmentationEngineJSONCompactEachRow(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", &fileSegmentationEngineJSONCompactEachRow);
|
auto get_file_segmentation_engine = [](size_t min_rows)
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", &fileSegmentationEngineJSONCompactEachRow);
|
{
|
||||||
|
return [min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
||||||
|
{
|
||||||
|
return fileSegmentationEngineJSONCompactEachRow(in, memory, min_chunk_size, min_rows);
|
||||||
|
};
|
||||||
|
};
|
||||||
|
|
||||||
|
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactEachRow", get_file_segmentation_engine);
|
||||||
|
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "JSONCompactStringsEachRow", get_file_segmentation_engine);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -2,6 +2,7 @@
|
|||||||
#include <IO/WriteBufferValidUTF8.h>
|
#include <IO/WriteBufferValidUTF8.h>
|
||||||
#include <Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h>
|
#include <Processors/Formats/Impl/JSONCompactEachRowRowOutputFormat.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
|
|
||||||
|
|
||||||
namespace DB
|
namespace DB
|
||||||
@ -101,20 +102,19 @@ void registerOutputFormatJSONCompactEachRow(FormatFactory & factory)
|
|||||||
{
|
{
|
||||||
for (bool yield_strings : {false, true})
|
for (bool yield_strings : {false, true})
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_output_creator = [yield_strings](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerOutputFormat(format_name, [=](
|
return [yield_strings, with_names, with_types](
|
||||||
WriteBuffer & buf,
|
WriteBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
const RowOutputFormatParams & params,
|
const RowOutputFormatParams & params,
|
||||||
const FormatSettings & format_settings)
|
const FormatSettings & format_settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, with_names, with_types, yield_strings);
|
return std::make_shared<JSONCompactEachRowRowOutputFormat>(buf, sample, params, format_settings, with_names, with_types, yield_strings);
|
||||||
});
|
};
|
||||||
factory.markOutputFormatSupportsParallelFormatting(format_name);
|
|
||||||
};
|
};
|
||||||
|
|
||||||
registerOutputFormatWithNamesAndTypes(yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", register_func);
|
registerOutputFormatWithNamesAndTypes(factory, yield_strings ? "JSONCompactStringsEachRow" : "JSONCompactEachRow", get_output_creator, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -5,6 +5,7 @@
|
|||||||
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
|
#include <Processors/Formats/Impl/TabSeparatedRowInputFormat.h>
|
||||||
#include <Formats/verbosePrintString.h>
|
#include <Formats/verbosePrintString.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
#include <DataTypes/DataTypeLowCardinality.h>
|
#include <DataTypes/DataTypeLowCardinality.h>
|
||||||
#include <DataTypes/Serializations/SerializationNullable.h>
|
#include <DataTypes/Serializations/SerializationNullable.h>
|
||||||
|
|
||||||
@ -221,25 +222,24 @@ void registerInputFormatTabSeparated(FormatFactory & factory)
|
|||||||
{
|
{
|
||||||
for (bool is_raw : {false, true})
|
for (bool is_raw : {false, true})
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_input_creator = [is_raw](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerInputFormat(format_name, [with_names, with_types, is_raw](
|
return [with_names, with_types, is_raw](
|
||||||
ReadBuffer & buf,
|
ReadBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
IRowInputFormat::Params params,
|
IRowInputFormat::Params params,
|
||||||
const FormatSettings & settings)
|
const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, std::move(params), with_names, with_types, is_raw, settings);
|
return std::make_shared<TabSeparatedRowInputFormat>(sample, buf, std::move(params), with_names, with_types, is_raw, settings);
|
||||||
});
|
};
|
||||||
};
|
};
|
||||||
|
|
||||||
registerInputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
|
registerInputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_input_creator);
|
||||||
registerInputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
|
registerInputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_input_creator);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
template <bool is_raw>
|
static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size, bool is_raw, size_t min_rows)
|
||||||
static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
|
||||||
{
|
{
|
||||||
bool need_more_data = true;
|
bool need_more_data = true;
|
||||||
char * pos = in.position();
|
char * pos = in.position();
|
||||||
@ -247,7 +247,7 @@ static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer
|
|||||||
|
|
||||||
while (loadAtPosition(in, memory, pos) && need_more_data)
|
while (loadAtPosition(in, memory, pos) && need_more_data)
|
||||||
{
|
{
|
||||||
if constexpr (is_raw)
|
if (is_raw)
|
||||||
pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end());
|
pos = find_first_symbols<'\r', '\n'>(pos, in.buffer().end());
|
||||||
else
|
else
|
||||||
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
|
pos = find_first_symbols<'\\', '\r', '\n'>(pos, in.buffer().end());
|
||||||
@ -269,7 +269,7 @@ static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer
|
|||||||
if (*pos == '\n')
|
if (*pos == '\n')
|
||||||
++number_of_rows;
|
++number_of_rows;
|
||||||
|
|
||||||
if (memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size)
|
if ((memory.size() + static_cast<size_t>(pos - in.position()) >= min_chunk_size) && number_of_rows >= min_rows)
|
||||||
need_more_data = false;
|
need_more_data = false;
|
||||||
++pos;
|
++pos;
|
||||||
}
|
}
|
||||||
@ -282,13 +282,28 @@ static std::pair<bool, size_t> fileSegmentationEngineTabSeparatedImpl(ReadBuffer
|
|||||||
|
|
||||||
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory)
|
void registerFileSegmentationEngineTabSeparated(FormatFactory & factory)
|
||||||
{
|
{
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSV", &fileSegmentationEngineTabSeparatedImpl<false>);
|
for (bool is_raw : {false, true})
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparated", &fileSegmentationEngineTabSeparatedImpl<false>);
|
{
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TSVRaw", &fileSegmentationEngineTabSeparatedImpl<true>);
|
auto get_file_segmentation_engine = [is_raw](size_t min_rows)
|
||||||
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, "TabSeparatedRaw", &fileSegmentationEngineTabSeparatedImpl<true>);
|
{
|
||||||
|
return [is_raw, min_rows](ReadBuffer & in, DB::Memory<> & memory, size_t min_chunk_size)
|
||||||
|
{
|
||||||
|
return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, is_raw, min_rows);
|
||||||
|
};
|
||||||
|
};
|
||||||
|
|
||||||
|
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_file_segmentation_engine);
|
||||||
|
registerFileSegmentationEngineForFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_file_segmentation_engine);
|
||||||
|
}
|
||||||
|
|
||||||
// We can use the same segmentation engine for TSKV.
|
// We can use the same segmentation engine for TSKV.
|
||||||
factory.registerFileSegmentationEngine("TSKV", &fileSegmentationEngineTabSeparatedImpl<false>);
|
factory.registerFileSegmentationEngine("TSKV", [](
|
||||||
|
ReadBuffer & in,
|
||||||
|
DB::Memory<> & memory,
|
||||||
|
size_t min_chunk_size)
|
||||||
|
{
|
||||||
|
return fileSegmentationEngineTabSeparatedImpl(in, memory, min_chunk_size, false, 0);
|
||||||
|
});
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -1,5 +1,6 @@
|
|||||||
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
|
#include <Processors/Formats/Impl/TabSeparatedRowOutputFormat.h>
|
||||||
#include <Formats/FormatFactory.h>
|
#include <Formats/FormatFactory.h>
|
||||||
|
#include <Formats/registerWithNamesAndTypes.h>
|
||||||
#include <IO/WriteHelpers.h>
|
#include <IO/WriteHelpers.h>
|
||||||
|
|
||||||
|
|
||||||
@ -78,21 +79,20 @@ void registerOutputFormatTabSeparated(FormatFactory & factory)
|
|||||||
{
|
{
|
||||||
for (bool is_raw : {false, true})
|
for (bool is_raw : {false, true})
|
||||||
{
|
{
|
||||||
auto register_func = [&](const String & format_name, bool with_names, bool with_types)
|
auto get_output_creator = [is_raw](bool with_names, bool with_types)
|
||||||
{
|
{
|
||||||
factory.registerOutputFormat(format_name, [=](
|
return [is_raw, with_names, with_types](
|
||||||
WriteBuffer & buf,
|
WriteBuffer & buf,
|
||||||
const Block & sample,
|
const Block & sample,
|
||||||
const RowOutputFormatParams & params,
|
const RowOutputFormatParams & params,
|
||||||
const FormatSettings & settings)
|
const FormatSettings & settings)
|
||||||
{
|
{
|
||||||
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, with_names, with_types, is_raw, params, settings);
|
return std::make_shared<TabSeparatedRowOutputFormat>(buf, sample, with_names, with_types, is_raw, params, settings);
|
||||||
});
|
};
|
||||||
factory.markOutputFormatSupportsParallelFormatting(format_name);
|
|
||||||
};
|
};
|
||||||
|
|
||||||
registerOutputFormatWithNamesAndTypes(is_raw ? "TSVRaw" : "TSV", register_func);
|
registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TSVRaw" : "TSV", get_output_creator, true);
|
||||||
registerOutputFormatWithNamesAndTypes(is_raw ? "TabSeparatedRaw" : "TabSeparated", register_func);
|
registerOutputFormatWithNamesAndTypes(factory, is_raw ? "TabSeparatedRaw" : "TabSeparated", get_output_creator, true);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -247,12 +247,6 @@ bool RowInputFormatWithNamesAndTypes::parseRowAndPrintDiagnosticInfo(MutableColu
|
|||||||
return parseRowEndWithDiagnosticInfo(out);
|
return parseRowEndWithDiagnosticInfo(out);
|
||||||
}
|
}
|
||||||
|
|
||||||
void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func)
|
|
||||||
{
|
|
||||||
register_func(base_format_name, false, false);
|
|
||||||
register_func(base_format_name + "WithNames", true, false);
|
|
||||||
register_func(base_format_name + "WithNamesAndTypes", true, true);
|
|
||||||
}
|
|
||||||
|
|
||||||
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
||||||
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine)
|
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine)
|
||||||
|
@ -63,13 +63,6 @@ private:
|
|||||||
std::unordered_map<String, size_t> column_indexes_by_names;
|
std::unordered_map<String, size_t> column_indexes_by_names;
|
||||||
};
|
};
|
||||||
|
|
||||||
using RegisterFormatWithNamesAndTypesFunc = std::function<void(
|
|
||||||
const String & format_name,
|
|
||||||
bool with_names,
|
|
||||||
bool with_types)>;
|
|
||||||
|
|
||||||
void registerInputFormatWithNamesAndTypes(const String & base_format_name, RegisterFormatWithNamesAndTypesFunc register_func);
|
|
||||||
|
|
||||||
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
void registerFileSegmentationEngineForFormatWithNamesAndTypes(
|
||||||
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine);
|
FormatFactory & factory, const String & base_format_name, FormatFactory::FileSegmentationEngine segmentation_engine);
|
||||||
|
|
||||||
|
@ -1,8 +1,14 @@
|
|||||||
1 text 2020-01-01
|
1 text 2020-01-01
|
||||||
1 text 2020-01-01
|
1 text 2020-01-01
|
||||||
1 text 2020-01-01
|
1 text 2020-01-01
|
||||||
|
1 text 2020-01-01
|
||||||
|
1 text 2020-01-01
|
||||||
|
1 text 2020-01-01
|
||||||
|
1 default 1970-01-01
|
||||||
1 default 1970-01-01
|
1 default 1970-01-01
|
||||||
1 1970-01-01
|
1 1970-01-01
|
||||||
|
1 1970-01-01
|
||||||
|
OK
|
||||||
1 default 1970-01-01
|
1 default 1970-01-01
|
||||||
OK
|
OK
|
||||||
OK
|
OK
|
||||||
|
@ -9,43 +9,60 @@ $CLICKHOUSE_CLIENT -q "DROP TABLE IF EXISTS test_02102"
|
|||||||
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02102 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()"
|
$CLICKHOUSE_CLIENT -q "CREATE TABLE test_02102 (x UInt32, y String DEFAULT 'default', z Date) engine=Memory()"
|
||||||
|
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames"
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames"
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' AS y, toDate('2020-01-01') AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=0 --input_format_with_types_use_header=0 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames"
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS y, toDate('2020-01-01') AS z, toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames"
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames"
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_defaults_for_omitted_fields=0 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNames" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNames" 2>&1 | grep -F -q "CANNOT_SKIP_UNKNOWN_FIELD" && echo 'OK' || echo 'FAIL'
|
||||||
|
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, [[1, 2, 3], [4, 5], []] as a FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_skip_unknown_fields=1 --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes"
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM test_02102"
|
||||||
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
$CLICKHOUSE_CLIENT -q "TRUNCATE TABLE test_02102"
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
$CLICKHOUSE_CLIENT -q "SELECT 'text' AS x, toDate('2020-01-01') AS y, toUInt32(1) AS z FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||||
|
|
||||||
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
$CLICKHOUSE_CLIENT -q "SELECT toUInt32(1) AS x, 'text' as z, toDate('2020-01-01') AS y FORMAT RowBinaryWithNamesAndTypes" | $CLICKHOUSE_CLIENT --input_format_with_names_use_header=1 --input_format_with_types_use_header=1 -q "INSERT INTO test_02102 FORMAT RowBinaryWithNamesAndTypes" 2>&1 | grep -F -q "INCORRECT_DATA" && echo 'OK' || echo 'FAIL'
|
||||||
|
@ -0,0 +1,80 @@
|
|||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
||||||
|
0 [0,1,2,3,4,5,6,7,8,9] 0
|
||||||
|
1 [0,1,2,3,4,5,6,7,8,9,10] 1
|
||||||
|
2 [0,1,2,3,4,5,6,7,8,9,10,11] 2
|
||||||
|
3 [0,1,2,3,4,5,6,7,8,9,10,11,12] 3
|
||||||
|
4 [0,1,2,3,4,5,6,7,8,9,10,11,12,13] 4
|
||||||
|
5 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14] 5
|
||||||
|
6 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15] 6
|
||||||
|
7 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16] 7
|
||||||
|
8 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17] 8
|
||||||
|
9 [0,1,2,3,4,5,6,7,8,9,10,11,12,13,14,15,16,17,18] 9
|
20
tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh
Executable file
20
tests/queries/0_stateless/02103_with_names_and_types_parallel_parsing.sh
Executable file
@ -0,0 +1,20 @@
|
|||||||
|
#!/usr/bin/env bash
|
||||||
|
|
||||||
|
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
|
||||||
|
# shellcheck source=../shell_config.sh
|
||||||
|
. "$CURDIR"/../shell_config.sh
|
||||||
|
|
||||||
|
USER_FILES_PATH=$(clickhouse-client --query "select _path,_file from file('nonexist.txt', 'CSV', 'val1 char')" 2>&1 | grep Exception | awk '{gsub("/nonexist.txt","",$9); print $9}')
|
||||||
|
|
||||||
|
DATA_FILE=$USER_FILES_PATH/test_02103.data
|
||||||
|
|
||||||
|
FORMATS=('TSVWithNames' 'TSVWithNamesAndTypes' 'TSVRawWithNames' 'TSVRawWithNamesAndTypes' 'CSVWithNames' 'CSVWithNamesAndTypes' 'JSONCompactEachRowWithNames' 'JSONCompactEachRowWithNamesAndTypes')
|
||||||
|
|
||||||
|
for format in "${FORMATS[@]}"
|
||||||
|
do
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT number, range(number + 10) AS array, toString(number) AS string FROM numbers(10) FORMAT $format" > $DATA_FILE
|
||||||
|
$CLICKHOUSE_CLIENT -q "SELECT * FROM file('test_02103.data', '$format', 'number UInt64, array Array(UInt64), string String') SETTINGS input_format_parallel_parsing=1, min_chunk_bytes_for_parallel_parsing=40"
|
||||||
|
done
|
||||||
|
|
||||||
|
rm $DATA_FILE
|
||||||
|
|
Loading…
Reference in New Issue
Block a user