Merge pull request #34351 from ClickHouse/small-improvements-2

Small improvements
This commit is contained in:
Maksim Kita 2022-02-06 13:36:37 +01:00 committed by GitHub
commit 971790fca5
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
35 changed files with 44 additions and 46 deletions

View File

@ -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)

View File

@ -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;

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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

View File

@ -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_);

View File

@ -23,7 +23,7 @@ private:
WriteBuffer & out;
};
class CapnProtoRowOutputFormat : public IRowOutputFormat
class CapnProtoRowOutputFormat final : public IRowOutputFormat
{
public:
CapnProtoRowOutputFormat(

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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(

View File

@ -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(

View File

@ -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(

View File

@ -5,7 +5,7 @@
namespace DB
{
class JSONEachRowWithProgressRowOutputFormat : public JSONEachRowRowOutputFormat
class JSONEachRowWithProgressRowOutputFormat final : public JSONEachRowRowOutputFormat
{
public:
using JSONEachRowRowOutputFormat::JSONEachRowRowOutputFormat;

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);

View File

@ -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(

View File

@ -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:
};
}

View File

@ -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:
};
}

View File

@ -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_);

View File

@ -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_);

View File

@ -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);

View File

@ -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_);

View File

@ -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);

View File

@ -15,7 +15,7 @@ namespace DB
class TemplateFormatReader;
class TemplateRowInputFormat : public RowInputFormatWithDiagnosticInfo
class TemplateRowInputFormat final : public RowInputFormatWithDiagnosticInfo
{
using EscapingRule = FormatSettings::EscapingRule;
public:

View File

@ -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_);

View File

@ -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_);

View File

@ -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_);