2018-06-10 19:22:49 +00:00
|
|
|
#pragma once
|
|
|
|
|
2021-10-16 08:28:10 +00:00
|
|
|
#include <Common/Allocator.h>
|
2020-02-03 10:02:52 +00:00
|
|
|
#include <Columns/IColumn.h>
|
2020-11-02 07:50:38 +00:00
|
|
|
#include <Formats/FormatSettings.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
#include <Interpreters/Context_fwd.h>
|
2021-12-15 11:30:57 +00:00
|
|
|
#include <IO/BufferWithOwnMemory.h>
|
2022-01-07 05:16:41 +00:00
|
|
|
#include <IO/CompressionMethod.h>
|
2023-03-13 19:29:59 +00:00
|
|
|
#include <IO/ParallelReadBuffer.h>
|
2021-10-02 07:13:14 +00:00
|
|
|
#include <base/types.h>
|
2021-12-15 11:30:57 +00:00
|
|
|
#include <Core/NamesAndTypes.h>
|
2021-04-10 23:33:54 +00:00
|
|
|
|
|
|
|
#include <boost/noncopyable.hpp>
|
2019-05-17 14:34:25 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <functional>
|
2019-05-17 14:34:25 +00:00
|
|
|
#include <memory>
|
2018-06-10 19:22:49 +00:00
|
|
|
#include <unordered_map>
|
|
|
|
|
2023-03-13 03:51:50 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
class Block;
|
2020-11-02 07:50:38 +00:00
|
|
|
struct Settings;
|
2020-11-07 08:53:39 +00:00
|
|
|
struct FormatFactorySettings;
|
2024-03-19 16:04:29 +00:00
|
|
|
struct ReadSettings;
|
2018-06-10 19:22:49 +00:00
|
|
|
|
|
|
|
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;
|
2022-10-28 16:41:10 +00:00
|
|
|
class IRowOutputFormat;
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2019-04-05 11:39:07 +00:00
|
|
|
struct RowInputFormatParams;
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2021-12-15 11:30:57 +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>;
|
2022-10-28 16:41:10 +00:00
|
|
|
using RowOutputFormatPtr = std::shared_ptr<IRowOutputFormat>;
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2021-10-16 08:28:10 +00:00
|
|
|
template <typename Allocator>
|
|
|
|
struct Memory;
|
|
|
|
|
2024-01-22 22:55:50 +00:00
|
|
|
FormatSettings getFormatSettings(const ContextPtr & context);
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2020-11-07 08:53:39 +00:00
|
|
|
template <typename T>
|
2024-01-22 22:55:50 +00:00
|
|
|
FormatSettings getFormatSettings(const ContextPtr & context, const T & settings);
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2021-10-19 09:58:10 +00:00
|
|
|
/** Allows to create an IInputFormat or IOutputFormat by the name of the format.
|
2018-06-10 19:22:49 +00:00
|
|
|
* Note: format and compression are independent things.
|
|
|
|
*/
|
2019-08-22 03:24:05 +00:00
|
|
|
class FormatFactory final : private boost::noncopyable
|
2018-06-10 19:22:49 +00:00
|
|
|
{
|
2019-05-23 11:15:18 +00:00
|
|
|
public:
|
2019-10-01 10:48:46 +00:00
|
|
|
/** Fast reading data from buffer and save result to memory.
|
2022-09-27 20:14:15 +00:00
|
|
|
* Reads at least `min_bytes` and some more until the end of the chunk, depends on the format.
|
|
|
|
* If `max_rows` is non-zero the function also stops after reading the `max_rows` number of rows
|
|
|
|
* (even if the `min_bytes` boundary isn't reached yet).
|
2022-05-09 19:13:02 +00:00
|
|
|
* Used in ParallelParsingInputFormat.
|
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,
|
2021-10-16 08:28:10 +00:00
|
|
|
DB::Memory<Allocator<false>> & memory,
|
2022-09-27 20:14:15 +00:00
|
|
|
size_t min_bytes,
|
|
|
|
size_t max_rows)>;
|
2019-10-01 10:48:46 +00:00
|
|
|
|
2023-11-21 03:22:12 +00:00
|
|
|
using FileSegmentationEngineCreator = std::function<FileSegmentationEngine(
|
|
|
|
const FormatSettings & settings)>;
|
|
|
|
|
2021-10-20 12:47:20 +00:00
|
|
|
private:
|
2023-03-28 20:28:28 +00:00
|
|
|
// On the input side, there are two kinds of formats:
|
|
|
|
// * InputCreator - formats parsed sequentially, e.g. CSV. Almost all formats are like this.
|
|
|
|
// FormatFactory uses ParallelReadBuffer to read in parallel, and ParallelParsingInputFormat
|
|
|
|
// to parse in parallel; the formats mostly don't need to worry about it.
|
|
|
|
// * RandomAccessInputCreator - column-oriented formats that require seeking back and forth in
|
|
|
|
// the file when reading. E.g. Parquet has metadata at the end of the file (needs to be read
|
|
|
|
// before we can parse any data), can skip columns by seeking in the file, and often reads
|
|
|
|
// many short byte ranges from the file. ParallelReadBuffer and ParallelParsingInputFormat
|
|
|
|
// are a poor fit. Instead, the format implementation is in charge of parallelizing both
|
|
|
|
// reading and parsing.
|
|
|
|
|
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
|
|
|
|
2023-03-20 07:55:44 +00:00
|
|
|
// Incompatible with FileSegmentationEngine.
|
2023-03-28 20:28:28 +00:00
|
|
|
using RandomAccessInputCreator = std::function<InputFormatPtr(
|
2023-05-05 03:11:51 +00:00
|
|
|
ReadBuffer & buf,
|
2023-03-13 19:29:59 +00:00
|
|
|
const Block & header,
|
|
|
|
const FormatSettings & settings,
|
|
|
|
const ReadSettings& read_settings,
|
|
|
|
bool is_remote_fs,
|
|
|
|
size_t max_download_threads,
|
|
|
|
size_t max_parsing_threads)>;
|
|
|
|
|
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 FormatSettings & settings)>;
|
|
|
|
|
2021-06-23 13:17:34 +00:00
|
|
|
/// 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)>;
|
|
|
|
|
2022-01-14 15:16:18 +00:00
|
|
|
/// Some formats can support append depending on settings.
|
|
|
|
/// The checker should return true if format support append.
|
|
|
|
using AppendSupportChecker = std::function<bool(const FormatSettings & settings)>;
|
2021-12-29 18:03:15 +00:00
|
|
|
|
2022-03-24 12:54:12 +00:00
|
|
|
using SchemaReaderCreator = std::function<SchemaReaderPtr(ReadBuffer & in, const FormatSettings & settings)>;
|
2021-12-15 11:30:57 +00:00
|
|
|
using ExternalSchemaReaderCreator = std::function<ExternalSchemaReaderPtr(const FormatSettings & settings)>;
|
|
|
|
|
2022-06-27 12:43:24 +00:00
|
|
|
/// Some formats can extract different schemas from the same source depending on
|
|
|
|
/// some settings. To process this case in schema cache we should add some additional
|
|
|
|
/// information to a cache key. This getter should return some string with information
|
|
|
|
/// about such settings. For example, for Protobuf format it's the path to the schema
|
|
|
|
/// and the name of the message.
|
|
|
|
using AdditionalInfoForSchemaCacheGetter = std::function<String(const FormatSettings & settings)>;
|
|
|
|
|
2023-09-11 14:55:37 +00:00
|
|
|
/// Some formats can support reading subset of columns depending on settings.
|
|
|
|
/// The checker should return true if format support append.
|
|
|
|
using SubsetOfColumnsSupportChecker = std::function<bool(const FormatSettings & settings)>;
|
|
|
|
|
2019-08-02 14:41:19 +00:00
|
|
|
struct Creators
|
|
|
|
{
|
2024-02-26 16:48:34 +00:00
|
|
|
String name;
|
2019-10-01 10:48:46 +00:00
|
|
|
InputCreator input_creator;
|
2023-03-28 20:28:28 +00:00
|
|
|
RandomAccessInputCreator random_access_input_creator;
|
2019-08-02 14:41:19 +00:00
|
|
|
OutputCreator output_creator;
|
2023-11-21 03:22:12 +00:00
|
|
|
FileSegmentationEngineCreator file_segmentation_engine_creator;
|
2021-12-15 11:30:57 +00:00
|
|
|
SchemaReaderCreator schema_reader_creator;
|
|
|
|
ExternalSchemaReaderCreator external_schema_reader_creator;
|
2020-10-06 14:02:01 +00:00
|
|
|
bool supports_parallel_formatting{false};
|
2023-04-29 02:29:51 +00:00
|
|
|
bool prefers_large_blocks{false};
|
2021-06-23 13:17:34 +00:00
|
|
|
NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker;
|
2022-01-14 15:16:18 +00:00
|
|
|
AppendSupportChecker append_support_checker;
|
2022-06-27 12:43:24 +00:00
|
|
|
AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter;
|
2023-09-11 14:55:37 +00:00
|
|
|
SubsetOfColumnsSupportChecker subset_of_columns_support_checker;
|
2019-08-02 14:41:19 +00:00
|
|
|
};
|
2018-06-10 19:22:49 +00:00
|
|
|
|
|
|
|
using FormatsDictionary = std::unordered_map<String, Creators>;
|
2022-01-07 05:16:41 +00:00
|
|
|
using FileExtensionFormats = std::unordered_map<String, String>;
|
2018-06-10 19:22:49 +00:00
|
|
|
|
|
|
|
public:
|
2019-08-22 03:24:05 +00:00
|
|
|
static FormatFactory & instance();
|
|
|
|
|
2023-05-05 03:11:51 +00:00
|
|
|
/// This has two tricks up its sleeve:
|
|
|
|
/// * Parallel reading.
|
|
|
|
/// To enable it, make sure `buf` is a SeekableReadBuffer implementing readBigAt().
|
|
|
|
/// * Parallel parsing.
|
2023-06-21 23:46:12 +00:00
|
|
|
/// `buf` must outlive the returned IInputFormat.
|
2020-05-18 10:00:22 +00:00
|
|
|
InputFormatPtr getInput(
|
2019-05-23 11:15:18 +00:00
|
|
|
const String & name,
|
|
|
|
ReadBuffer & buf,
|
|
|
|
const Block & sample,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2019-05-23 11:15:18 +00:00
|
|
|
UInt64 max_block_size,
|
2023-03-20 07:55:44 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings = std::nullopt,
|
2023-05-05 03:11:51 +00:00
|
|
|
std::optional<size_t> max_parsing_threads = std::nullopt,
|
2023-03-20 07:55:44 +00:00
|
|
|
std::optional<size_t> max_download_threads = std::nullopt,
|
2023-05-05 03:11:51 +00:00
|
|
|
// affects things like buffer sizes and parallel reading
|
|
|
|
bool is_remote_fs = false,
|
|
|
|
// allows to do: buf -> parallel read -> decompression,
|
|
|
|
// because parallel read after decompression is not possible
|
2023-10-17 00:00:07 +00:00
|
|
|
CompressionMethod compression = CompressionMethod::None,
|
|
|
|
bool need_only_count = false) 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(
|
2021-04-10 23:33:54 +00:00
|
|
|
const String & name,
|
|
|
|
WriteBuffer & buf,
|
|
|
|
const Block & sample,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2020-12-30 03:07:30 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
|
|
|
|
|
2019-08-20 11:17:57 +00:00
|
|
|
OutputFormatPtr getOutputFormat(
|
2021-04-10 23:33:54 +00:00
|
|
|
const String & name,
|
|
|
|
WriteBuffer & buf,
|
|
|
|
const Block & sample,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2021-12-03 11:42:46 +00:00
|
|
|
const std::optional<FormatSettings> & _format_settings = std::nullopt) const;
|
|
|
|
|
|
|
|
String getContentType(
|
|
|
|
const String & name,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2020-11-07 08:53:39 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
|
2019-02-19 18:41:18 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
SchemaReaderPtr getSchemaReader(
|
|
|
|
const String & name,
|
|
|
|
ReadBuffer & buf,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2021-12-15 11:30:57 +00:00
|
|
|
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
|
|
|
|
|
|
|
|
ExternalSchemaReaderPtr getExternalSchemaReader(
|
|
|
|
const String & name,
|
2024-01-22 22:55:50 +00:00
|
|
|
const ContextPtr & context,
|
2021-12-15 11:30:57 +00:00
|
|
|
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);
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2023-11-21 03:22:12 +00:00
|
|
|
void registerFileSegmentationEngineCreator(const String & name, FileSegmentationEngineCreator file_segmentation_engine_creator);
|
|
|
|
|
2021-06-23 13:17:34 +00:00
|
|
|
void registerNonTrivialPrefixAndSuffixChecker(const String & name, NonTrivialPrefixAndSuffixChecker non_trivial_prefix_and_suffix_checker);
|
|
|
|
|
2022-01-14 15:16:18 +00:00
|
|
|
void registerAppendSupportChecker(const String & name, AppendSupportChecker append_support_checker);
|
2021-12-29 18:03:15 +00:00
|
|
|
|
2022-01-14 15:16:18 +00:00
|
|
|
/// If format always doesn't support append, you can use this method instead of
|
|
|
|
/// registerAppendSupportChecker with append_support_checker that always returns true.
|
2022-01-24 13:27:04 +00:00
|
|
|
void markFormatHasNoAppendSupport(const String & name);
|
2021-12-29 18:03:15 +00:00
|
|
|
|
2024-01-22 22:55:50 +00:00
|
|
|
bool checkIfFormatSupportAppend(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
|
2021-12-29 18:03:15 +00:00
|
|
|
|
2021-10-11 16:11:50 +00:00
|
|
|
/// Register format by its name.
|
2023-03-28 20:28:28 +00:00
|
|
|
void registerInputFormat(const String & name, InputCreator input_creator);
|
|
|
|
void registerRandomAccessInputFormat(const String & name, RandomAccessInputCreator input_creator);
|
2021-10-11 16:11:50 +00:00
|
|
|
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);
|
2024-01-22 22:55:50 +00:00
|
|
|
String getFormatFromFileName(String file_name);
|
|
|
|
std::optional<String> tryGetFormatFromFileName(String file_name);
|
2022-01-24 18:41:44 +00:00
|
|
|
String getFormatFromFileDescriptor(int fd);
|
2024-01-22 22:55:50 +00:00
|
|
|
std::optional<String> tryGetFormatFromFileDescriptor(int fd);
|
2022-01-07 05:16:41 +00:00
|
|
|
|
2021-12-15 11:30:57 +00:00
|
|
|
/// 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);
|
2023-04-29 02:29:51 +00:00
|
|
|
void markOutputFormatPrefersLargeBlocks(const String & name);
|
2021-03-30 21:25:37 +00:00
|
|
|
|
2023-09-11 14:55:37 +00:00
|
|
|
void markFormatSupportsSubsetOfColumns(const String & name);
|
|
|
|
void registerSubsetOfColumnsSupportChecker(const String & name, SubsetOfColumnsSupportChecker subset_of_columns_support_checker);
|
|
|
|
bool checkIfFormatSupportsSubsetOfColumns(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_ = std::nullopt) const;
|
2020-10-06 14:02:01 +00:00
|
|
|
|
2022-05-20 14:57:27 +00:00
|
|
|
bool checkIfFormatHasSchemaReader(const String & name) const;
|
|
|
|
bool checkIfFormatHasExternalSchemaReader(const String & name) const;
|
|
|
|
bool checkIfFormatHasAnySchemaReader(const String & name) const;
|
2023-04-29 02:29:51 +00:00
|
|
|
bool checkIfOutputFormatPrefersLargeBlocks(const String & name) const;
|
2021-12-15 11:30:57 +00:00
|
|
|
|
2024-01-22 22:55:50 +00:00
|
|
|
bool checkParallelizeOutputAfterReading(const String & name, const ContextPtr & context) const;
|
2023-05-05 04:18:46 +00:00
|
|
|
|
2022-06-27 12:43:24 +00:00
|
|
|
void registerAdditionalInfoForSchemaCacheGetter(const String & name, AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter);
|
2024-01-22 22:55:50 +00:00
|
|
|
String getAdditionalInfoForSchemaCache(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
|
2022-06-27 12:43:24 +00:00
|
|
|
|
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;
|
2018-07-20 10:00:56 +00:00
|
|
|
}
|
|
|
|
|
2024-01-22 22:55:50 +00:00
|
|
|
std::vector<String> getAllInputFormats() const;
|
|
|
|
|
2021-09-16 17:18:34 +00:00
|
|
|
bool isInputFormat(const String & name) const;
|
|
|
|
bool isOutputFormat(const String & name) const;
|
|
|
|
|
2022-05-23 12:48:48 +00:00
|
|
|
/// Check that format with specified name exists and throw an exception otherwise.
|
|
|
|
void checkFormatName(const String & name) const;
|
2024-02-26 16:48:34 +00:00
|
|
|
bool exists(const String & name) const;
|
2022-05-23 12:48:48 +00:00
|
|
|
|
2018-06-10 19:22:49 +00:00
|
|
|
private:
|
2019-08-02 14:41:19 +00:00
|
|
|
FormatsDictionary dict;
|
2024-02-26 16:48:34 +00:00
|
|
|
FileExtensionFormats file_extension_formats;
|
2018-06-10 19:22:49 +00:00
|
|
|
|
2019-08-02 14:41:19 +00:00
|
|
|
const Creators & getCreators(const String & name) const;
|
2024-02-26 16:48:34 +00:00
|
|
|
Creators & getOrCreateCreators(const String & name);
|
2021-12-15 11:30:57 +00:00
|
|
|
|
2023-05-05 03:11:51 +00:00
|
|
|
// Creates a ReadBuffer to give to an input format. Returns nullptr if we should use `buf` directly.
|
|
|
|
std::unique_ptr<ReadBuffer> wrapReadBufferIfNeeded(
|
|
|
|
ReadBuffer & buf,
|
2023-03-28 20:28:28 +00:00
|
|
|
CompressionMethod compression,
|
|
|
|
const Creators & creators,
|
|
|
|
const FormatSettings & format_settings,
|
|
|
|
const Settings & settings,
|
2023-05-05 03:11:51 +00:00
|
|
|
bool is_remote_fs,
|
2023-03-28 20:28:28 +00:00
|
|
|
size_t max_download_threads) const;
|
2018-06-10 19:22:49 +00:00
|
|
|
};
|
|
|
|
|
|
|
|
}
|