mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-11 18:14:03 +00:00
Merge pull request #34351 from ClickHouse/small-improvements-2
Small improvements
This commit is contained in:
commit
971790fca5
@ -15,7 +15,8 @@ IRowOutputFormat::IRowOutputFormat(const Block & header, WriteBuffer & out_, con
|
||||
, types(header.getDataTypes())
|
||||
, params(params_)
|
||||
{
|
||||
serializations.reserve(types.size());
|
||||
num_columns = types.size();
|
||||
serializations.reserve(num_columns);
|
||||
for (const auto & type : types)
|
||||
serializations.push_back(type->getDefaultSerialization());
|
||||
}
|
||||
@ -68,8 +69,6 @@ void IRowOutputFormat::consumeExtremes(DB::Chunk chunk)
|
||||
|
||||
void IRowOutputFormat::write(const Columns & columns, size_t row_num)
|
||||
{
|
||||
size_t num_columns = columns.size();
|
||||
|
||||
writeRowStartDelimiter();
|
||||
|
||||
for (size_t i = 0; i < num_columns; ++i)
|
||||
|
@ -57,6 +57,7 @@ protected:
|
||||
virtual void writeAfterExtremes() {}
|
||||
virtual void finalizeImpl() override {} /// Write something after resultset, totals end extremes.
|
||||
|
||||
size_t num_columns;
|
||||
DataTypes types;
|
||||
Serializations serializations;
|
||||
Params params;
|
||||
|
@ -115,7 +115,7 @@ private:
|
||||
std::map<avro::Name, SkipFn> symbolic_skip_fn_map;
|
||||
};
|
||||
|
||||
class AvroRowInputFormat : public IRowInputFormat
|
||||
class AvroRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
AvroRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_);
|
||||
@ -137,7 +137,7 @@ private:
|
||||
/// 2. SchemaRegistry: schema cache (schema_id -> schema)
|
||||
/// 3. AvroConfluentRowInputFormat: deserializer cache (schema_id -> AvroDeserializer)
|
||||
/// This is needed because KafkaStorage creates a new instance of InputFormat per a batch of messages
|
||||
class AvroConfluentRowInputFormat : public IRowInputFormat
|
||||
class AvroConfluentRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
AvroConfluentRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_, const FormatSettings & format_settings_);
|
||||
|
@ -43,7 +43,7 @@ private:
|
||||
std::unique_ptr<AvroSerializerTraits> traits;
|
||||
};
|
||||
|
||||
class AvroRowOutputFormat : public IRowOutputFormat
|
||||
class AvroRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
AvroRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & settings_);
|
||||
|
@ -17,7 +17,7 @@ class ReadBuffer;
|
||||
|
||||
/** A stream for inputting data in a binary line-by-line format.
|
||||
*/
|
||||
class BinaryRowInputFormat : public RowInputFormatWithNamesAndTypes
|
||||
class BinaryRowInputFormat final : public RowInputFormatWithNamesAndTypes
|
||||
{
|
||||
public:
|
||||
BinaryRowInputFormat(ReadBuffer & in_, Block header, Params params_, bool with_names_, bool with_types_, const FormatSettings & format_settings_);
|
||||
@ -30,7 +30,7 @@ public:
|
||||
std::string getDiagnosticInfo() override { return {}; }
|
||||
};
|
||||
|
||||
class BinaryFormatReader : public FormatWithNamesAndTypesReader
|
||||
class BinaryFormatReader final : public FormatWithNamesAndTypesReader
|
||||
{
|
||||
public:
|
||||
BinaryFormatReader(ReadBuffer & in_, const FormatSettings & format_settings_);
|
||||
|
@ -14,7 +14,7 @@ class WriteBuffer;
|
||||
|
||||
/** A stream for outputting data in a binary line-by-line format.
|
||||
*/
|
||||
class BinaryRowOutputFormat: public IRowOutputFormat
|
||||
class BinaryRowOutputFormat final: public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
BinaryRowOutputFormat(WriteBuffer & out_, const Block & header, bool with_names_, bool with_types_, const RowOutputFormatParams & params_);
|
||||
|
@ -14,7 +14,7 @@ class WriteBuffer;
|
||||
/** The stream for outputting data in csv format.
|
||||
* Does not conform with https://tools.ietf.org/html/rfc4180 because it uses LF, not CR LF.
|
||||
*/
|
||||
class CSVRowOutputFormat : public IRowOutputFormat
|
||||
class CSVRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
/** with_names - output in the first line a header with column names
|
||||
|
@ -20,7 +20,7 @@ class ReadBuffer;
|
||||
* The schema in this case cannot be compiled in, so it uses a runtime schema parser.
|
||||
* See https://capnproto.org/cxx.html
|
||||
*/
|
||||
class CapnProtoRowInputFormat : public IRowInputFormat
|
||||
class CapnProtoRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
CapnProtoRowInputFormat(ReadBuffer & in_, Block header, Params params_, const FormatSchemaInfo & info, const FormatSettings & format_settings_);
|
||||
|
@ -23,7 +23,7 @@ private:
|
||||
WriteBuffer & out;
|
||||
};
|
||||
|
||||
class CapnProtoRowOutputFormat : public IRowOutputFormat
|
||||
class CapnProtoRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
CapnProtoRowOutputFormat(
|
||||
|
@ -8,7 +8,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class CustomSeparatedRowInputFormat : public RowInputFormatWithNamesAndTypes
|
||||
class CustomSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes
|
||||
{
|
||||
public:
|
||||
CustomSeparatedRowInputFormat(
|
||||
@ -35,7 +35,7 @@ private:
|
||||
bool ignore_spaces;
|
||||
};
|
||||
|
||||
class CustomSeparatedFormatReader : public FormatWithNamesAndTypesReader
|
||||
class CustomSeparatedFormatReader final : public FormatWithNamesAndTypesReader
|
||||
{
|
||||
public:
|
||||
CustomSeparatedFormatReader(PeekableReadBuffer & buf_, bool ignore_spaces_, const FormatSettings & format_settings_);
|
||||
|
@ -8,7 +8,7 @@ namespace DB
|
||||
|
||||
class WriteBuffer;
|
||||
|
||||
class CustomSeparatedRowOutputFormat : public IRowOutputFormat
|
||||
class CustomSeparatedRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
CustomSeparatedRowOutputFormat(const Block & header_, WriteBuffer & out_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_, bool with_names_, bool with_types_);
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
|
||||
/// A stream for input data in Hive Text format.
|
||||
/// Parallel parsing is disabled currently.
|
||||
class HiveTextRowInputFormat : public CSVRowInputFormat
|
||||
class HiveTextRowInputFormat final : public CSVRowInputFormat
|
||||
{
|
||||
public:
|
||||
HiveTextRowInputFormat(const Block & header_, ReadBuffer & in_, const Params & params_, const FormatSettings & format_settings_);
|
||||
@ -24,7 +24,7 @@ private:
|
||||
const Block & header_, std::unique_ptr<PeekableReadBuffer> buf_, const Params & params_, const FormatSettings & format_settings_);
|
||||
};
|
||||
|
||||
class HiveTextFormatReader : public CSVFormatReader
|
||||
class HiveTextFormatReader final : public CSVFormatReader
|
||||
{
|
||||
public:
|
||||
HiveTextFormatReader(std::unique_ptr<PeekableReadBuffer> buf_, const FormatSettings & format_settings_);
|
||||
|
@ -15,7 +15,7 @@ class ReadBuffer;
|
||||
/// Each JSON object is parsed as a whole to string.
|
||||
/// This format can only parse a table with single field of type String.
|
||||
|
||||
class JSONAsStringRowInputFormat : public IRowInputFormat
|
||||
class JSONAsStringRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
JSONAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
|
@ -19,7 +19,7 @@ class ReadBuffer;
|
||||
* - JSONCompactStringsEachRowWithNamesAndTypes
|
||||
*
|
||||
*/
|
||||
class JSONCompactEachRowRowInputFormat : public RowInputFormatWithNamesAndTypes
|
||||
class JSONCompactEachRowRowInputFormat final : public RowInputFormatWithNamesAndTypes
|
||||
{
|
||||
public:
|
||||
JSONCompactEachRowRowInputFormat(
|
||||
@ -38,7 +38,7 @@ private:
|
||||
void syncAfterError() override;
|
||||
};
|
||||
|
||||
class JSONCompactEachRowFormatReader : public FormatWithNamesAndTypesReader
|
||||
class JSONCompactEachRowFormatReader final : public FormatWithNamesAndTypesReader
|
||||
{
|
||||
public:
|
||||
JSONCompactEachRowFormatReader(ReadBuffer & in_, bool yield_strings_, const FormatSettings & format_settings_);
|
||||
|
@ -12,7 +12,7 @@ namespace DB
|
||||
/** The stream for outputting data in JSON format, by object per line.
|
||||
* Does not validate UTF-8.
|
||||
*/
|
||||
class JSONCompactEachRowRowOutputFormat : public IRowOutputFormat
|
||||
class JSONCompactEachRowRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
JSONCompactEachRowRowOutputFormat(
|
||||
|
@ -13,7 +13,7 @@ struct FormatSettings;
|
||||
|
||||
/** The stream for outputting data in the JSONCompact- formats.
|
||||
*/
|
||||
class JSONCompactRowOutputFormat : public JSONRowOutputFormat
|
||||
class JSONCompactRowOutputFormat final : public JSONRowOutputFormat
|
||||
{
|
||||
public:
|
||||
JSONCompactRowOutputFormat(
|
||||
|
@ -18,7 +18,7 @@ class ReadBuffer;
|
||||
* Fields can be listed in any order (including, in different lines there may be different order),
|
||||
* and some fields may be missing.
|
||||
*/
|
||||
class JSONEachRowRowInputFormat : public IRowInputFormat
|
||||
class JSONEachRowRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
JSONEachRowRowInputFormat(
|
||||
|
@ -5,7 +5,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class JSONEachRowWithProgressRowOutputFormat : public JSONEachRowRowOutputFormat
|
||||
class JSONEachRowWithProgressRowOutputFormat final : public JSONEachRowRowOutputFormat
|
||||
{
|
||||
public:
|
||||
using JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat;
|
||||
|
@ -14,7 +14,7 @@ class ReadBuffer;
|
||||
/// Each Line object is parsed as a whole to string.
|
||||
/// This format can only parse a table with single field of type String.
|
||||
|
||||
class LineAsStringRowInputFormat : public IRowInputFormat
|
||||
class LineAsStringRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
LineAsStringRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
|
@ -9,7 +9,7 @@ namespace DB
|
||||
|
||||
class ReadBuffer;
|
||||
|
||||
class MarkdownRowOutputFormat : public IRowOutputFormat
|
||||
class MarkdownRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
MarkdownRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
|
||||
|
@ -55,7 +55,7 @@ private:
|
||||
std::stack<Info> info_stack;
|
||||
};
|
||||
|
||||
class MsgPackRowInputFormat : public IRowInputFormat
|
||||
class MsgPackRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
MsgPackRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
|
@ -15,7 +15,7 @@
|
||||
namespace DB
|
||||
{
|
||||
|
||||
class MsgPackRowOutputFormat : public IRowOutputFormat
|
||||
class MsgPackRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
MsgPackRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_);
|
||||
|
@ -26,7 +26,7 @@ class ProtobufSerializer;
|
||||
* INSERT INTO table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
|
||||
* where schema is the name of "schema.proto" file specifying protobuf schema.
|
||||
*/
|
||||
class ProtobufRowInputFormat : public IRowInputFormat
|
||||
class ProtobufRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
ProtobufRowInputFormat(ReadBuffer & in_, const Block & header_, const Params & params_, const FormatSchemaInfo & schema_info_, bool with_length_delimiter_);
|
||||
|
@ -26,7 +26,7 @@ struct FormatSettings;
|
||||
* SELECT * from table FORMAT Protobuf SETTINGS format_schema = 'schema:Message'
|
||||
* where schema is the name of "schema.proto" file specifying protobuf schema.
|
||||
*/
|
||||
class ProtobufRowOutputFormat : public IRowOutputFormat
|
||||
class ProtobufRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
ProtobufRowOutputFormat(
|
||||
|
@ -13,7 +13,7 @@ class ReadBuffer;
|
||||
/// This format slurps all input data into single value.
|
||||
/// This format can only parse a table with single field of type String or similar.
|
||||
|
||||
class RawBLOBRowInputFormat : public IRowInputFormat
|
||||
class RawBLOBRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
RawBLOBRowInputFormat(const Block & header_, ReadBuffer & in_, Params params_);
|
||||
@ -34,4 +34,3 @@ public:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -24,7 +24,7 @@ class WriteBuffer;
|
||||
*
|
||||
* If you are output more than one value, the output format is ambiguous and you may not be able to read data back.
|
||||
*/
|
||||
class RawBLOBRowOutputFormat : public IRowOutputFormat
|
||||
class RawBLOBRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
RawBLOBRowOutputFormat(
|
||||
@ -39,4 +39,3 @@ private:
|
||||
};
|
||||
|
||||
}
|
||||
|
||||
|
@ -48,7 +48,7 @@ private:
|
||||
/// (according to format_regexp_escaping_rule setting). If the regexp did not match the line,
|
||||
/// if format_regexp_skip_unmatched is 1, the line is silently skipped, if the setting is 0, exception will be thrown.
|
||||
|
||||
class RegexpRowInputFormat : public IRowInputFormat
|
||||
class RegexpRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
RegexpRowInputFormat(ReadBuffer & in_, const Block & header_, Params params_, const FormatSettings & format_settings_);
|
||||
|
@ -21,7 +21,7 @@ class ReadBuffer;
|
||||
* An equal sign can be escaped in the field name.
|
||||
* Also, as an additional element there may be a useless tskv fragment - it needs to be ignored.
|
||||
*/
|
||||
class TSKVRowInputFormat : public IRowInputFormat
|
||||
class TSKVRowInputFormat final : public IRowInputFormat
|
||||
{
|
||||
public:
|
||||
TSKVRowInputFormat(ReadBuffer & in_, Block header_, Params params_, const FormatSettings & format_settings_);
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
* TSKV is similar to TabSeparated, but before every value, its name and equal sign are specified: name=value.
|
||||
* This format is very inefficient.
|
||||
*/
|
||||
class TSKVRowOutputFormat: public TabSeparatedRowOutputFormat
|
||||
class TSKVRowOutputFormat final : public TabSeparatedRowOutputFormat
|
||||
{
|
||||
public:
|
||||
TSKVRowOutputFormat(WriteBuffer & out_, const Block & header, const RowOutputFormatParams & params_, const FormatSettings & format_settings);
|
||||
|
@ -11,7 +11,7 @@ namespace DB
|
||||
|
||||
/** A stream to input data in tsv format.
|
||||
*/
|
||||
class TabSeparatedRowInputFormat : public RowInputFormatWithNamesAndTypes
|
||||
class TabSeparatedRowInputFormat final : public RowInputFormatWithNamesAndTypes
|
||||
{
|
||||
public:
|
||||
/** with_names - the first line is the header with the names of the columns
|
||||
@ -28,7 +28,7 @@ private:
|
||||
bool isGarbageAfterField(size_t, ReadBuffer::Position pos) override { return *pos != '\n' && *pos != '\t'; }
|
||||
};
|
||||
|
||||
class TabSeparatedFormatReader : public FormatWithNamesAndTypesReader
|
||||
class TabSeparatedFormatReader final : public FormatWithNamesAndTypesReader
|
||||
{
|
||||
public:
|
||||
TabSeparatedFormatReader(ReadBuffer & in_, const FormatSettings & format_settings, bool is_raw_);
|
||||
|
@ -34,10 +34,10 @@ public:
|
||||
|
||||
protected:
|
||||
void writeField(const IColumn & column, const ISerialization & serialization, size_t row_num) override;
|
||||
void writeFieldDelimiter() override;
|
||||
void writeFieldDelimiter() override final;
|
||||
void writeRowEndDelimiter() override;
|
||||
void writeBeforeTotals() override;
|
||||
void writeBeforeExtremes() override;
|
||||
void writeBeforeTotals() override final;
|
||||
void writeBeforeExtremes() override final;
|
||||
|
||||
void writePrefix() override;
|
||||
void writeLine(const std::vector<String> & values);
|
||||
|
@ -15,7 +15,7 @@ namespace DB
|
||||
|
||||
class TemplateFormatReader;
|
||||
|
||||
class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
|
||||
class TemplateRowInputFormat final : public RowInputFormatWithDiagnosticInfo
|
||||
{
|
||||
using EscapingRule = FormatSettings::EscapingRule;
|
||||
public:
|
||||
|
@ -12,7 +12,7 @@ class WriteBuffer;
|
||||
|
||||
/** A stream for outputting data in the VALUES format (as in the INSERT request).
|
||||
*/
|
||||
class ValuesRowOutputFormat : public IRowOutputFormat
|
||||
class ValuesRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
ValuesRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
|
||||
|
@ -15,7 +15,7 @@ class Context;
|
||||
/** Stream to output data in format "each value in separate row".
|
||||
* Usable to show few rows with many columns.
|
||||
*/
|
||||
class VerticalRowOutputFormat : public IRowOutputFormat
|
||||
class VerticalRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
VerticalRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
|
||||
|
@ -13,7 +13,7 @@ namespace DB
|
||||
|
||||
/** A stream for outputting data in XML format.
|
||||
*/
|
||||
class XMLRowOutputFormat : public IRowOutputFormat
|
||||
class XMLRowOutputFormat final : public IRowOutputFormat
|
||||
{
|
||||
public:
|
||||
XMLRowOutputFormat(WriteBuffer & out_, const Block & header_, const RowOutputFormatParams & params_, const FormatSettings & format_settings_);
|
||||
|
Loading…
Reference in New Issue
Block a user