ClickHouse/src/Formats/FormatFactory.h

221 lines
7.8 KiB
C++
Raw Normal View History

#pragma once
#include <Common/Allocator.h>
2020-02-03 10:02:52 +00:00
#include <Columns/IColumn.h>
#include <Formats/FormatSettings.h>
#include <Interpreters/Context_fwd.h>
#include <IO/BufferWithOwnMemory.h>
2022-01-07 05:16:41 +00:00
#include <IO/CompressionMethod.h>
2021-10-02 07:13:14 +00:00
#include <base/types.h>
#include <Core/NamesAndTypes.h>
#include <boost/noncopyable.hpp>
#include <functional>
#include <memory>
#include <unordered_map>
namespace DB
{
class Block;
struct Settings;
struct FormatFactorySettings;
class ReadBuffer;
class WriteBuffer;
2019-03-26 18:28:37 +00:00
class IProcessor;
using ProcessorPtr = std::shared_ptr<IProcessor>;
2019-02-19 18:41:18 +00:00
class IInputFormat;
class IOutputFormat;
struct RowInputFormatParams;
struct RowOutputFormatParams;
2019-02-19 18:41:18 +00:00
class ISchemaReader;
class IExternalSchemaReader;
using SchemaReaderPtr = std::shared_ptr<ISchemaReader>;
using ExternalSchemaReaderPtr = std::shared_ptr<IExternalSchemaReader>;
2019-02-19 18:41:18 +00:00
using InputFormatPtr = std::shared_ptr<IInputFormat>;
using OutputFormatPtr = std::shared_ptr<IOutputFormat>;
template <typename Allocator>
struct Memory;
2021-06-01 12:20:52 +00:00
FormatSettings getFormatSettings(ContextPtr context);
template <typename T>
2021-06-01 12:20:52 +00:00
FormatSettings getFormatSettings(ContextPtr context, const T & settings);
2021-10-19 09:58:10 +00:00
/** Allows to create an IInputFormat or IOutputFormat by the name of the format.
* Note: format and compression are independent things.
*/
class FormatFactory final : private boost::noncopyable
{
public:
2019-06-14 17:19:02 +00:00
/// This callback allows to perform some additional actions after reading a single row.
/// It's initial purpose was to extract payload for virtual columns from Kafka Consumer ReadBuffer.
using ReadCallback = std::function<void()>;
2019-10-01 10:48:46 +00:00
/** Fast reading data from buffer and save result to memory.
2019-11-18 19:25:17 +00:00
* Reads at least min_chunk_bytes and some more until the end of the chunk, depends on the format.
* Used in ParallelParsingBlockInputStream.
2019-10-01 10:48:46 +00:00
*/
2020-11-30 16:42:41 +00:00
using FileSegmentationEngine = std::function<std::pair<bool, size_t>(
2019-10-01 10:48:46 +00:00
ReadBuffer & buf,
DB::Memory<Allocator<false>> & memory,
2019-11-18 19:25:17 +00:00
size_t min_chunk_bytes)>;
2019-10-01 10:48:46 +00:00
/// This callback allows to perform some additional actions after writing a single row.
/// It's initial purpose was to flush Kafka message for each row.
2020-02-03 10:02:52 +00:00
using WriteCallback = std::function<void(
const Columns & columns,
size_t row)>;
2021-10-20 12:47:20 +00:00
private:
2021-11-10 17:29:52 +00:00
using InputCreator = std::function<InputFormatPtr(
ReadBuffer & buf,
const Block & header,
const RowInputFormatParams & params,
const FormatSettings & settings)>;
2019-02-19 18:41:18 +00:00
2021-10-11 16:11:50 +00:00
using OutputCreator = std::function<OutputFormatPtr(
2019-02-19 18:41:18 +00:00
WriteBuffer & buf,
const Block & sample,
const RowOutputFormatParams & params,
2019-02-19 18:41:18 +00:00
const FormatSettings & settings)>;
/// Some input formats can have non trivial readPrefix() and readSuffix(),
/// so in some cases there is no possibility to use parallel parsing.
/// The checker should return true if parallel parsing should be disabled.
using NonTrivialPrefixAndSuffixChecker = std::function<bool(ReadBuffer & buf)>;
/// Some formats can have suffix after data depending on settings.
/// The checker should return true if format will write some suffix after data.
using SuffixChecker = std::function<bool(const FormatSettings & settings)>;
using SchemaReaderCreator = std::function<SchemaReaderPtr(ReadBuffer & in, const FormatSettings & settings, ContextPtr context)>;
using ExternalSchemaReaderCreator = std::function<ExternalSchemaReaderPtr(const FormatSettings & settings)>;
2019-08-02 14:41:19 +00:00
struct Creators
{
2019-10-01 10:48:46 +00:00
InputCreator input_creator;
2019-08-02 14:41:19 +00:00
OutputCreator output_creator;
2019-10-01 10:48:46 +00:00
FileSegmentationEngine file_segmentation_engine;
SchemaReaderCreator schema_reader_creator;
ExternalSchemaReaderCreator external_schema_reader_creator;
2020-10-06 14:02:01 +00:00
bool supports_parallel_formatting{false};
2021-03-30 21:25:37 +00:00
bool is_column_oriented{false};
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
SuffixChecker suffix_checker;
2019-08-02 14:41:19 +00:00
};
using FormatsDictionary = std::unordered_map<String, Creators>;
2022-01-07 05:16:41 +00:00
using FileExtensionFormats = std::unordered_map<String, String>;
public:
static FormatFactory & instance();
2020-05-18 10:00:22 +00:00
InputFormatPtr getInput(
const String & name,
ReadBuffer & buf,
const Block & sample,
2021-06-01 12:20:52 +00:00
ContextPtr context,
UInt64 max_block_size,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
InputFormatPtr getInputFormat(
const String & name,
ReadBuffer & buf,
const Block & sample,
2021-06-01 12:20:52 +00:00
ContextPtr context,
UInt64 max_block_size,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
2019-02-19 18:41:18 +00:00
2020-12-30 03:07:30 +00:00
/// Checks all preconditions. Returns ordinary format if parallel formatting cannot be done.
OutputFormatPtr getOutputFormatParallelIfPossible(
const String & name,
WriteBuffer & buf,
const Block & sample,
2021-06-01 12:20:52 +00:00
ContextPtr context,
WriteCallback callback = {},
2020-12-30 03:07:30 +00:00
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
OutputFormatPtr getOutputFormat(
const String & name,
WriteBuffer & buf,
const Block & sample,
2021-06-01 12:20:52 +00:00
ContextPtr context,
WriteCallback callback = {},
const std::optional<FormatSettings> & _format_settings = std::nullopt) const;
String getContentType(
const String & name,
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
2019-02-19 18:41:18 +00:00
SchemaReaderPtr getSchemaReader(
const String & name,
ReadBuffer & buf,
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
ExternalSchemaReaderPtr getExternalSchemaReader(
const String & name,
ContextPtr context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
2019-10-01 10:48:46 +00:00
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker);
void registerSuffixChecker(const String & name, SuffixChecker suffix_checker);
/// If format always contains suffix, you an use this method instead of
/// registerSuffixChecker with suffix_checker that always returns true.
void markFormatWithSuffix(const String & name);
bool checkIfFormatHasSuffix(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
2021-10-11 16:11:50 +00:00
/// Register format by its name.
void registerInputFormat(const String & name, InputCreator input_creator);
void registerOutputFormat(const String & name, OutputCreator output_creator);
2019-02-19 18:41:18 +00:00
2022-01-07 05:16:41 +00:00
/// Register file extension for format
void registerFileExtension(const String & extension, const String & format_name);
String getFormatFromFileName(String file_name);
/// Register schema readers for format its name.
void registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator);
void registerExternalSchemaReader(const String & name, ExternalSchemaReaderCreator external_schema_reader_creator);
2020-10-06 14:02:01 +00:00
void markOutputFormatSupportsParallelFormatting(const String & name);
2021-03-30 21:25:37 +00:00
void markFormatAsColumnOriented(const String & name);
bool checkIfFormatIsColumnOriented(const String & name);
2020-10-06 14:02:01 +00:00
bool checkIfFormatHasSchemaReader(const String & name);
bool checkIfFormatHasExternalSchemaReader(const String & name);
bool checkIfFormatHasAnySchemaReader(const String & name);
2019-08-02 14:41:19 +00:00
const FormatsDictionary & getAllFormats() const
2018-07-20 15:59:11 +00:00
{
2019-08-02 14:41:19 +00:00
return dict;
}
2021-09-16 17:18:34 +00:00
bool isInputFormat(const String & name) const;
bool isOutputFormat(const String & name) const;
private:
2019-08-02 14:41:19 +00:00
FormatsDictionary dict;
2022-01-07 05:16:41 +00:00
FileExtensionFormats file_extension_formats;
2019-08-02 14:41:19 +00:00
const Creators & getCreators(const String & name) const;
};
}