Try to detect file format automatically during schema inference if it's unknown

This commit is contained in:
avogar 2024-01-22 22:55:50 +00:00
parent f5227ec98e
commit 617cc514b7
78 changed files with 2433 additions and 763 deletions

View File

@ -13,7 +13,7 @@ can control it.
Schema inference is used when ClickHouse needs to read the data in a specific data format and the structure is unknown.
## Table functions [file](../sql-reference/table-functions/file.md), [s3](../sql-reference/table-functions/s3.md), [url](../sql-reference/table-functions/url.md), [hdfs](../sql-reference/table-functions/hdfs.md).
## Table functions [file](../sql-reference/table-functions/file.md), [s3](../sql-reference/table-functions/s3.md), [url](../sql-reference/table-functions/url.md), [hdfs](../sql-reference/table-functions/hdfs.md), [azureBlobStorage](../sql-reference/table-functions/azureBlobStorage.md).
These table functions have the optional argument `structure` with the structure of input data. If this argument is not specified or set to `auto`, the structure will be inferred from the data.
@ -55,7 +55,7 @@ DESCRIBE file('hobbies.jsonl')
└─────────┴─────────────────────────┴──────────────┴────────────────────┴─────────┴──────────────────┴────────────────┘
```
## Table engines [File](../engines/table-engines/special/file.md), [S3](../engines/table-engines/integrations/s3.md), [URL](../engines/table-engines/special/url.md), [HDFS](../engines/table-engines/integrations/hdfs.md)
## Table engines [File](../engines/table-engines/special/file.md), [S3](../engines/table-engines/integrations/s3.md), [URL](../engines/table-engines/special/url.md), [HDFS](../engines/table-engines/integrations/hdfs.md), [azureBlobStorage](./engines/table-engines/integrations/azureBlobStorage.md)
If the list of columns is not specified in `CREATE TABLE` query, the structure of the table will be inferred automatically from the data.
@ -1061,7 +1061,7 @@ $$)
└──────────────┴───────────────┘
```
## Values {#values}
### Values {#values}
In Values format ClickHouse extracts column value from the row and then parses it using
the recursive parser similar to how literals are parsed.
@ -1986,3 +1986,45 @@ Note:
- As some of the files may not contain some columns from the resulting schema, union mode is supported only for formats that support reading subset of columns (like JSONEachRow, Parquet, TSVWithNames, etc) and won't work for other formats (like CSV, TSV, JSONCompactEachRow, etc).
- If ClickHouse cannot infer the schema from one of the files, the exception will be thrown.
- If you have a lot of files, reading schema from all of them can take a lot of time.
## Automatic format detection {#autimatic-format-detection}
If data format is not specified and cannot be determined by the file extension, ClickHouse will try to detect the file format by its content.
**Examples:**
Let's say we have `data` with the next content:
`data1`:
```
"a","b"
1,"Data1"
2,"Data2"
3,"Data3"
```
We can inspect and query this file without specifying format or structure:
```sql
:) desc file(data);
```
```text
┌─name─┬─type─────────────┐
│ a │ Nullable(Int64) │
│ b │ Nullable(String) │
└──────┴──────────────────┘
```
```sql
:) select * from file(data);
```
```text
┌─a─┬─b─────┐
│ 1 │ Data1 │
│ 2 │ Data2 │
│ 3 │ Data3 │
└───┴───────┘
```
**Note:** ClickHouse can detect only some subset of formats and this detection takes some time, it's always better to specify the format explicitly.

View File

@ -348,7 +348,7 @@ std::string LocalServer::getInitialCreateTableQuery()
/// Use regular file
auto file_name = config().getString("table-file");
table_file = quoteString(file_name);
format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name, false);
format_from_file_name = FormatFactory::instance().getFormatFromFileName(file_name);
}
auto data_format = backQuoteIfNeed(

View File

@ -1310,7 +1310,7 @@ try
throw ErrnoException(ErrorCodes::CANNOT_SEEK_THROUGH_FILE, "Input must be seekable file (it will be read twice)");
SingleReadBufferIterator read_buffer_iterator(std::move(file));
schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, false, context_const);
schema_columns = readSchemaFromFormat(input_format, {}, read_buffer_iterator, context_const);
}
else
{

View File

@ -1508,7 +1508,7 @@ void ClientBase::sendData(Block & sample, const ColumnsDescription & columns_des
String current_format = parsed_insert_query->format;
if (current_format.empty())
current_format = FormatFactory::instance().getFormatFromFileName(in_file, true);
current_format = FormatFactory::instance().getFormatFromFileName(in_file);
/// Create temporary storage file, to support globs and parallel reading
/// StorageFile doesn't support ephemeral/materialized/alias columns.

View File

@ -592,6 +592,7 @@
M(710, FAULT_INJECTED) \
M(711, FILECACHE_ACCESS_DENIED) \
M(712, TOO_MANY_MATERIALIZED_VIEWS) \
M(713, CANNOT_DETECT_FORMAT) \
\
M(999, KEEPER_EXCEPTION) \
M(1000, POCO_EXCEPTION) \

View File

@ -146,9 +146,18 @@ StoragePtr DatabaseFilesystem::getTableImpl(const String & name, ContextPtr cont
if (!checkTableFilePath(table_path, context_, throw_on_error))
return {};
auto format = FormatFactory::instance().getFormatFromFileName(table_path, throw_on_error);
if (format.empty())
return {};
String format;
if (throw_on_error)
{
format = FormatFactory::instance().getFormatFromFileName(table_path);
}
else
{
auto format_maybe = FormatFactory::instance().tryGetFormatFromFileName(table_path);
if (!format_maybe)
return {};
format = *format_maybe;
}
auto ast_function_ptr = makeASTFunction("file", std::make_shared<ASTLiteral>(table_path), std::make_shared<ASTLiteral>(format));

View File

@ -39,7 +39,7 @@ const FormatFactory::Creators & FormatFactory::getCreators(const String & name)
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown format {}", name);
}
FormatSettings getFormatSettings(ContextPtr context)
FormatSettings getFormatSettings(const ContextPtr & context)
{
const auto & settings = context->getSettingsRef();
@ -47,7 +47,7 @@ FormatSettings getFormatSettings(ContextPtr context)
}
template <typename Settings>
FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
FormatSettings getFormatSettings(const ContextPtr & context, const Settings & settings)
{
FormatSettings format_settings;
@ -253,16 +253,16 @@ FormatSettings getFormatSettings(ContextPtr context, const Settings & settings)
return format_settings;
}
template FormatSettings getFormatSettings<FormatFactorySettings>(ContextPtr context, const FormatFactorySettings & settings);
template FormatSettings getFormatSettings<FormatFactorySettings>(const ContextPtr & context, const FormatFactorySettings & settings);
template FormatSettings getFormatSettings<Settings>(ContextPtr context, const Settings & settings);
template FormatSettings getFormatSettings<Settings>(const ContextPtr & context, const Settings & settings);
InputFormatPtr FormatFactory::getInput(
const String & name,
ReadBuffer & _buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
UInt64 max_block_size,
const std::optional<FormatSettings> & _format_settings,
std::optional<size_t> _max_parsing_threads,
@ -425,7 +425,7 @@ std::unique_ptr<ReadBuffer> FormatFactory::wrapReadBufferIfNeeded(
return res;
}
static void addExistingProgressToOutputFormat(OutputFormatPtr format, ContextPtr context)
static void addExistingProgressToOutputFormat(OutputFormatPtr format, const ContextPtr & context)
{
auto element_id = context->getProcessListElementSafe();
if (element_id)
@ -444,7 +444,7 @@ OutputFormatPtr FormatFactory::getOutputFormatParallelIfPossible(
const String & name,
WriteBuffer & buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & output_getter = getCreators(name).output_creator;
@ -482,7 +482,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(
const String & name,
WriteBuffer & buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & output_getter = getCreators(name).output_creator;
@ -516,7 +516,7 @@ OutputFormatPtr FormatFactory::getOutputFormat(
String FormatFactory::getContentType(
const String & name,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & output_getter = getCreators(name).output_creator;
@ -535,7 +535,7 @@ String FormatFactory::getContentType(
SchemaReaderPtr FormatFactory::getSchemaReader(
const String & name,
ReadBuffer & buf,
ContextPtr & context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & schema_reader_creator = dict.at(name).schema_reader_creator;
@ -551,7 +551,7 @@ SchemaReaderPtr FormatFactory::getSchemaReader(
ExternalSchemaReaderPtr FormatFactory::getExternalSchemaReader(
const String & name,
ContextPtr & context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings) const
{
const auto & external_schema_reader_creator = dict.at(name).external_schema_reader_creator;
@ -605,7 +605,7 @@ void FormatFactory::markFormatHasNoAppendSupport(const String & name)
registerAppendSupportChecker(name, [](const FormatSettings &){ return false; });
}
bool FormatFactory::checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_)
bool FormatFactory::checkIfFormatSupportAppend(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_)
{
auto format_settings = format_settings_ ? *format_settings_ : getFormatSettings(context);
auto & append_support_checker = dict[name].append_support_checker;
@ -628,10 +628,10 @@ void FormatFactory::registerFileExtension(const String & extension, const String
file_extension_formats[boost::to_lower_copy(extension)] = format_name;
}
String FormatFactory::getFormatFromFileName(String file_name, bool throw_if_not_found)
std::optional<String> FormatFactory::tryGetFormatFromFileName(String file_name)
{
if (file_name == "stdin")
return getFormatFromFileDescriptor(STDIN_FILENO);
return tryGetFormatFromFileDescriptor(STDIN_FILENO);
CompressionMethod compression_method = chooseCompressionMethod(file_name, "");
if (CompressionMethod::None != compression_method)
@ -643,43 +643,53 @@ String FormatFactory::getFormatFromFileName(String file_name, bool throw_if_not_
auto pos = file_name.find_last_of('.');
if (pos == String::npos)
{
if (throw_if_not_found)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the file format by it's extension");
return "";
}
return std::nullopt;
String file_extension = file_name.substr(pos + 1, String::npos);
boost::algorithm::to_lower(file_extension);
auto it = file_extension_formats.find(file_extension);
if (it == file_extension_formats.end())
{
if (throw_if_not_found)
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the file format by it's extension");
return "";
}
return std::nullopt;
return it->second;
}
String FormatFactory::getFormatFromFileDescriptor(int fd)
String FormatFactory::getFormatFromFileName(String file_name)
{
if (auto format = tryGetFormatFromFileName(file_name))
return *format;
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the format of the file {} by it's extension", file_name);
}
std::optional<String> FormatFactory::tryGetFormatFromFileDescriptor(int fd)
{
#ifdef OS_LINUX
std::string proc_path = fmt::format("/proc/self/fd/{}", fd);
char file_path[PATH_MAX] = {'\0'};
if (readlink(proc_path.c_str(), file_path, sizeof(file_path) - 1) != -1)
return getFormatFromFileName(file_path, false);
return "";
return tryGetFormatFromFileName(file_path);
return std::nullopt;
#elif defined(OS_DARWIN)
char file_path[PATH_MAX] = {'\0'};
if (fcntl(fd, F_GETPATH, file_path) != -1)
return getFormatFromFileName(file_path, false);
return "";
return tryGetFormatFromFileName(file_path, false);
return std::nullopt;
#else
(void)fd;
return "";
return std::nullopt;
#endif
}
String FormatFactory::getFormatFromFileDescriptor(int fd)
{
if (auto format = tryGetFormatFromFileDescriptor(fd))
return *format;
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Cannot determine the format of the data by the file descriptor {}", fd);
}
void FormatFactory::registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine)
{
auto & target = dict[name].file_segmentation_engine_creator;
@ -765,7 +775,7 @@ void FormatFactory::registerAdditionalInfoForSchemaCacheGetter(
target = std::move(additional_info_for_schema_cache_getter);
}
String FormatFactory::getAdditionalInfoForSchemaCache(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_)
String FormatFactory::getAdditionalInfoForSchemaCache(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_)
{
const auto & additional_info_getter = getCreators(name).additional_info_for_schema_cache_getter;
if (!additional_info_getter)
@ -810,7 +820,7 @@ bool FormatFactory::checkIfOutputFormatPrefersLargeBlocks(const String & name) c
return target.prefers_large_blocks;
}
bool FormatFactory::checkParallelizeOutputAfterReading(const String & name, ContextPtr context) const
bool FormatFactory::checkParallelizeOutputAfterReading(const String & name, const ContextPtr & context) const
{
if (name == "Parquet" && context->getSettingsRef().input_format_parquet_preserve_order)
return false;
@ -825,6 +835,18 @@ void FormatFactory::checkFormatName(const String & name) const
throw Exception(ErrorCodes::UNKNOWN_FORMAT, "Unknown format {}", name);
}
std::vector<String> FormatFactory::getAllInputFormats() const
{
std::vector<String> input_formats;
for (const auto & [format_name, creators] : dict)
{
if (creators.input_creator || creators.random_access_input_creator)
input_formats.push_back(format_name);
}
return input_formats;
}
FormatFactory & FormatFactory::instance()
{
static FormatFactory ret;

View File

@ -48,10 +48,10 @@ using RowOutputFormatPtr = std::shared_ptr<IRowOutputFormat>;
template <typename Allocator>
struct Memory;
FormatSettings getFormatSettings(ContextPtr context);
FormatSettings getFormatSettings(const ContextPtr & context);
template <typename T>
FormatSettings getFormatSettings(ContextPtr context, const T & settings);
FormatSettings getFormatSettings(const ContextPtr & context, const T & settings);
/** Allows to create an IInputFormat or IOutputFormat by the name of the format.
* Note: format and compression are independent things.
@ -161,7 +161,7 @@ public:
const String & name,
ReadBuffer & buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
UInt64 max_block_size,
const std::optional<FormatSettings> & format_settings = std::nullopt,
std::optional<size_t> max_parsing_threads = std::nullopt,
@ -178,30 +178,30 @@ public:
const String & name,
WriteBuffer & buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
OutputFormatPtr getOutputFormat(
const String & name,
WriteBuffer & buf,
const Block & sample,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & _format_settings = std::nullopt) const;
String getContentType(
const String & name,
ContextPtr context,
const ContextPtr & context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
SchemaReaderPtr getSchemaReader(
const String & name,
ReadBuffer & buf,
ContextPtr & context,
const ContextPtr & context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
ExternalSchemaReaderPtr getExternalSchemaReader(
const String & name,
ContextPtr & context,
const ContextPtr & context,
const std::optional<FormatSettings> & format_settings = std::nullopt) const;
void registerFileSegmentationEngine(const String & name, FileSegmentationEngine file_segmentation_engine);
@ -216,7 +216,7 @@ public:
/// registerAppendSupportChecker with append_support_checker that always returns true.
void markFormatHasNoAppendSupport(const String & name);
bool checkIfFormatSupportAppend(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
bool checkIfFormatSupportAppend(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
/// Register format by its name.
void registerInputFormat(const String & name, InputCreator input_creator);
@ -225,8 +225,10 @@ public:
/// Register file extension for format
void registerFileExtension(const String & extension, const String & format_name);
String getFormatFromFileName(String file_name, bool throw_if_not_found = false);
String getFormatFromFileName(String file_name);
std::optional<String> tryGetFormatFromFileName(String file_name);
String getFormatFromFileDescriptor(int fd);
std::optional<String> tryGetFormatFromFileDescriptor(int fd);
/// Register schema readers for format its name.
void registerSchemaReader(const String & name, SchemaReaderCreator schema_reader_creator);
@ -244,16 +246,18 @@ public:
bool checkIfFormatHasAnySchemaReader(const String & name) const;
bool checkIfOutputFormatPrefersLargeBlocks(const String & name) const;
bool checkParallelizeOutputAfterReading(const String & name, ContextPtr context) const;
bool checkParallelizeOutputAfterReading(const String & name, const ContextPtr & context) const;
void registerAdditionalInfoForSchemaCacheGetter(const String & name, AdditionalInfoForSchemaCacheGetter additional_info_for_schema_cache_getter);
String getAdditionalInfoForSchemaCache(const String & name, ContextPtr context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
String getAdditionalInfoForSchemaCache(const String & name, const ContextPtr & context, const std::optional<FormatSettings> & format_settings_ = std::nullopt);
const FormatsDictionary & getAllFormats() const
{
return dict;
}
std::vector<String> getAllInputFormats() const;
bool isInputFormat(const String & name) const;
bool isOutputFormat(const String & name) const;

View File

@ -4,6 +4,7 @@
#include <Common/assert_cast.h>
#include <IO/WithFileSize.h>
#include <IO/EmptyReadBuffer.h>
#include <IO/PeekableReadBuffer.h>
namespace DB
{
@ -14,7 +15,9 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int ONLY_NULLS_WHILE_READING_SCHEMA;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int CANNOT_DETECT_FORMAT;
extern const int TYPE_MISMATCH;
extern const int LOGICAL_ERROR;
}
static std::optional<NamesAndTypesList> getOrderedColumnsList(const NamesAndTypesList & columns_list, const Names & columns_order_hint)
@ -43,48 +46,86 @@ bool isRetryableSchemaInferenceError(int code)
return code == ErrorCodes::EMPTY_DATA_PASSED || code == ErrorCodes::ONLY_NULLS_WHILE_READING_SCHEMA;
}
ColumnsDescription readSchemaFromFormat(
const String & format_name,
/// Order of formats to try in automatic format detection.
/// If we can successfully detect some format, we won't try next ones.
static const std::vector<String> & getFormatsOrderForDetection()
{
static const std::vector<String> formats_order =
{
"Parquet",
"ORC",
"Arrow",
"ArrowStream",
"Avro",
"AvroConfluent",
"Npy",
"Native",
"BSONEachRow",
"JSONCompact",
"Values",
"TSKV",
"JSONObjectEachRow",
"JSONColumns",
"JSONCompactColumns",
"JSONCompact",
"JSON",
};
return formats_order;
}
/// The set of similar formats to try in automatic format detection.
/// We will try all formats from this set and then choose the best one
/// according to inferred schema.
static const std::vector<String> & getSimilarFormatsSetForDetection()
{
static const std::vector<String> formats_order =
{
"TSV",
"CSV",
};
return formats_order;
}
std::pair<ColumnsDescription, String> readSchemaFromFormatImpl(
std::optional<String> format_name,
const std::optional<FormatSettings> & format_settings,
IReadBufferIterator & read_buffer_iterator,
bool retry,
ContextPtr & context,
std::unique_ptr<ReadBuffer> & buf)
const ContextPtr & context)
try
{
NamesAndTypesList names_and_types;
SchemaInferenceMode mode = context->getSettingsRef().schema_inference_mode;
if (mode == SchemaInferenceMode::UNION && !FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format_name, context, format_settings))
if (format_name && mode == SchemaInferenceMode::UNION && !FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(*format_name, context, format_settings))
{
String additional_message;
/// Better exception message for WithNames(AndTypes) formats.
if (format_name.ends_with("WithNames") || format_name.ends_with("WithNamesAndTypes"))
if (format_name->ends_with("WithNames") || format_name->ends_with("WithNamesAndTypes"))
additional_message = " (formats -WithNames(AndTypes) support reading subset of columns only when setting input_format_with_names_use_header is enabled)";
throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION schema inference mode is not supported for format {}, because it doesn't support reading subset of columns{}", format_name, additional_message);
throw Exception(ErrorCodes::BAD_ARGUMENTS, "UNION schema inference mode is not supported for format {}, because it doesn't support reading subset of columns{}", *format_name, additional_message);
}
if (FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format_name))
if (format_name && FormatFactory::instance().checkIfFormatHasExternalSchemaReader(*format_name))
{
auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(format_name, context, format_settings);
auto external_schema_reader = FormatFactory::instance().getExternalSchemaReader(*format_name, context, format_settings);
try
{
names_and_types = external_schema_reader->readSchema();
return {ColumnsDescription(external_schema_reader->readSchema()), *format_name};
}
catch (Exception & e)
{
e.addMessage(
fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
fmt::format("The table structure cannot be extracted from a {} format file. You can specify the structure manually", *format_name));
throw;
}
}
else if (FormatFactory::instance().checkIfFormatHasSchemaReader(format_name))
{
if (mode == SchemaInferenceMode::UNION)
retry = false;
if (!format_name || FormatFactory::instance().checkIfFormatHasSchemaReader(*format_name))
{
IReadBufferIterator::Data iterator_data;
std::vector<std::pair<NamesAndTypesList, String>> schemas_for_union_mode;
std::optional<ColumnsDescription> cached_columns;
std::string exception_messages;
SchemaReaderPtr schema_reader;
size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference
@ -94,45 +135,71 @@ try
size_t iterations = 0;
while (true)
{
/// When we finish working with current buffer we should put it back to iterator.
SCOPE_EXIT(if (iterator_data.buf) read_buffer_iterator.setPreviousReadBuffer(std::move(iterator_data.buf)));
bool is_eof = false;
try
{
read_buffer_iterator.setPreviousReadBuffer(std::move(buf));
std::tie(buf, cached_columns) = read_buffer_iterator.next();
if (cached_columns)
iterator_data = read_buffer_iterator.next();
/// Read buffer iterator can determine the data format if it's unknown.
/// For example by scanning schema cache or by finding new file with format extension.
if (!format_name && iterator_data.format_name)
{
format_name = *iterator_data.format_name;
read_buffer_iterator.setFormatName(*iterator_data.format_name);
}
if (iterator_data.cached_columns)
{
/// If we have schema in cache, we must also know the format.
if (!format_name)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Schema from cache was returned, but format name is unknown");
if (mode == SchemaInferenceMode::DEFAULT)
return *cached_columns;
schemas_for_union_mode.emplace_back(cached_columns->getAll(), read_buffer_iterator.getLastFileName());
{
read_buffer_iterator.setResultingSchema(*iterator_data.cached_columns);
return {*iterator_data.cached_columns, *format_name};
}
schemas_for_union_mode.emplace_back(iterator_data.cached_columns->getAll(), read_buffer_iterator.getLastFileName());
continue;
}
if (!buf)
if (!iterator_data.buf)
break;
/// We just want to check for eof, but eof() can be pretty expensive.
/// So we use getFileSize() when available, which has better worst case.
/// (For remote files, typically eof() would read 1 MB from S3, which may be much
/// more than what the schema reader and even data reader will read).
auto size = tryGetFileSizeFromReadBuffer(*buf);
auto size = tryGetFileSizeFromReadBuffer(*iterator_data.buf);
if (size.has_value())
is_eof = *size == 0;
else
is_eof = buf->eof();
is_eof = iterator_data.buf->eof();
}
catch (Exception & e)
{
e.addMessage(
fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
if (format_name)
e.addMessage(fmt::format("The table structure cannot be extracted from a {} format file. You can specify the structure manually", *format_name));
else
e.addMessage("The data format cannot be detected by the contents of the files. You can specify the format manually");
throw;
}
catch (...)
{
auto exception_message = getCurrentExceptionMessage(false);
if (format_name)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file:\n{}\nYou can specify the structure manually",
*format_name,
exception_message);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file:\n{}\nYou can specify the structure manually",
format_name,
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files:\n{}\nYou can specify the format manually",
exception_message);
}
@ -140,91 +207,218 @@ try
if (is_eof)
{
auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name);
String exception_message;
if (format_name)
exception_message = fmt::format("The table structure cannot be extracted from a {} format file: the file is empty", *format_name);
else
exception_message = fmt::format("The data format cannot be detected by the contents of the files: the file is empty");
if (!retry)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message);
if (mode == SchemaInferenceMode::UNION)
{
if (!format_name)
throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files: the file is empty. You can specify the format manually");
exception_messages += "\n" + exception_message;
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message);
}
if (!exception_messages.empty())
exception_messages += "\n";
exception_messages += exception_message;
continue;
}
try
if (format_name)
{
schema_reader = FormatFactory::instance().getSchemaReader(format_name, *buf, context, format_settings);
schema_reader->setMaxRowsAndBytesToRead(max_rows_to_read, max_bytes_to_read);
names_and_types = schema_reader->readSchema();
auto num_rows = schema_reader->readNumberOrRows();
if (num_rows)
read_buffer_iterator.setNumRowsToLastFile(*num_rows);
/// In default mode, we finish when schema is inferred successfully from any file.
if (mode == SchemaInferenceMode::DEFAULT)
break;
if (!names_and_types.empty())
read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types));
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName());
}
catch (...)
{
auto exception_message = getCurrentExceptionMessage(false);
if (schema_reader && mode == SchemaInferenceMode::DEFAULT)
try
{
size_t rows_read = schema_reader->getNumRowsRead();
assert(rows_read <= max_rows_to_read);
max_rows_to_read -= schema_reader->getNumRowsRead();
size_t bytes_read = buf->count();
/// We could exceed max_bytes_to_read a bit to complete row parsing.
max_bytes_to_read -= std::min(bytes_read, max_bytes_to_read);
if (rows_read != 0 && (max_rows_to_read == 0 || max_bytes_to_read == 0))
{
exception_message += "\nTo increase the maximum number of rows/bytes to read for structure determination, use setting "
"input_format_max_rows_to_read_for_schema_inference/input_format_max_bytes_to_read_for_schema_inference";
schema_reader = FormatFactory::instance().getSchemaReader(*format_name, *iterator_data.buf, context, format_settings);
schema_reader->setMaxRowsAndBytesToRead(max_rows_to_read, max_bytes_to_read);
names_and_types = schema_reader->readSchema();
auto num_rows = schema_reader->readNumberOrRows();
if (num_rows)
read_buffer_iterator.setNumRowsToLastFile(*num_rows);
if (iterations > 1)
/// In default mode, we finish when schema is inferred successfully from any file.
if (mode == SchemaInferenceMode::DEFAULT)
break;
if (!names_and_types.empty())
read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types));
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName());
}
catch (...)
{
auto exception_message = getCurrentExceptionMessage(false);
if (schema_reader && mode == SchemaInferenceMode::DEFAULT)
{
size_t rows_read = schema_reader->getNumRowsRead();
assert(rows_read <= max_rows_to_read);
max_rows_to_read -= schema_reader->getNumRowsRead();
size_t bytes_read = iterator_data.buf->count();
/// We could exceed max_bytes_to_read a bit to complete row parsing.
max_bytes_to_read -= std::min(bytes_read, max_bytes_to_read);
if (rows_read != 0 && (max_rows_to_read == 0 || max_bytes_to_read == 0))
{
exception_messages += "\n" + exception_message;
exception_message
+= "\nTo increase the maximum number of rows/bytes to read for structure determination, use setting "
"input_format_max_rows_to_read_for_schema_inference/input_format_max_bytes_to_read_for_schema_inference";
if (!exception_messages.empty())
exception_messages += "\n";
exception_messages += exception_message;
break;
}
retry = false;
}
}
if (!retry || !isRetryableSchemaInferenceError(getCurrentExceptionCode()))
{
try
{
throw;
}
catch (Exception & e)
{
e.addMessage(fmt::format(
"Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
throw;
}
catch (...)
if (mode == SchemaInferenceMode::UNION || !isRetryableSchemaInferenceError(getCurrentExceptionCode()))
{
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file. "
"The table structure cannot be extracted from a {} format file. "
"Error: {}. You can specify the structure manually",
format_name,
*format_name,
exception_message);
}
if (!exception_messages.empty())
exception_messages += "\n";
exception_messages += exception_message;
}
}
else
{
/// If the format is unknown we try some formats in order and try to apply their schema readers.
/// If we can successfully infer the schema in some format, most likely we can use this format to read this data.
/// If read_buffer_iterator supports recreation of last buffer, we will recreate it for
/// each format. Otherwise we will use PeekableReadBuffer and will rollback to the
/// beginning of the file before each format. Using PeekableReadBuffer can lead
/// to high memory usage as it will save all the read data from the beginning of the file,
/// especially it will be noticeable for formats like Parquet/ORC/Arrow that do seeks to the
/// end of file.
std::unique_ptr<PeekableReadBuffer> peekable_buf;
bool support_buf_recreation = read_buffer_iterator.supportsLastReadBufferRecreation();
if (!support_buf_recreation)
{
peekable_buf = std::make_unique<PeekableReadBuffer>(*iterator_data.buf);
peekable_buf->setCheckpoint();
}
/// First, try some formats in order. If we successfully inferred the schema for any format,
/// we will use this format.
for (const auto & format_to_detect : getFormatsOrderForDetection())
{
try
{
schema_reader = FormatFactory::instance().getSchemaReader(format_to_detect, support_buf_recreation ? *iterator_data.buf : *peekable_buf, context, format_settings);
schema_reader->setMaxRowsAndBytesToRead(max_rows_to_read, max_bytes_to_read);
names_and_types = schema_reader->readSchema();
if (names_and_types.empty())
continue;
/// We successfully inferred schema from this file using current format.
format_name = format_to_detect;
read_buffer_iterator.setFormatName(format_to_detect);
auto num_rows = schema_reader->readNumberOrRows();
if (num_rows)
read_buffer_iterator.setNumRowsToLastFile(*num_rows);
break;
}
catch (...)
{
/// We failed to infer the schema for this format.
/// Recreate read buffer or rollback to the beginning of the data
/// before trying next format.
if (support_buf_recreation)
{
read_buffer_iterator.setPreviousReadBuffer(std::move(iterator_data.buf));
iterator_data.buf = read_buffer_iterator.recreateLastReadBuffer();
}
else
{
peekable_buf->rollbackToCheckpoint();
}
}
}
exception_messages += "\n" + exception_message;
/// If no format was detected from first set of formats, we try second set.
/// In this set formats are similar and it can happen that data matches some of them.
/// We try to infer schema for all of the formats from this set and then choose the best
/// one according to the inferred schema.
if (!format_name)
{
std::unordered_map<String, NamesAndTypesList> format_to_schema;
for (const auto & format_to_detect : getSimilarFormatsSetForDetection())
{
try
{
schema_reader = FormatFactory::instance().getSchemaReader(
format_to_detect, support_buf_recreation ? *iterator_data.buf : *peekable_buf, context, format_settings);
schema_reader->setMaxRowsAndBytesToRead(max_rows_to_read, max_bytes_to_read);
auto tmp_names_and_types = schema_reader->readSchema();
/// If schema was inferred successfully for this format, remember it and try next format.
if (!tmp_names_and_types.empty())
format_to_schema[format_to_detect] = tmp_names_and_types;
}
catch (...) // NOLINT(bugprone-empty-catch)
{
/// Try next format.
}
if (support_buf_recreation)
{
read_buffer_iterator.setPreviousReadBuffer(std::move(iterator_data.buf));
iterator_data.buf = read_buffer_iterator.recreateLastReadBuffer();
}
else
{
peekable_buf->rollbackToCheckpoint();
}
}
/// We choose the format with larger number of columns in inferred schema.
size_t max_number_of_columns = 0;
for (const auto & [format_to_detect, schema] : format_to_schema )
{
if (schema.size() > max_number_of_columns)
{
names_and_types = schema;
format_name = format_to_detect;
max_number_of_columns = schema.size();
}
}
if (format_name)
read_buffer_iterator.setFormatName(*format_name);
}
if (mode == SchemaInferenceMode::UNION)
{
/// For UNION mode we need to know the schema of each file,
/// if we failed to detect the format, we failed to detect the schema of this file
/// in any format. It doesn't make sense to continue.
if (!format_name)
throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually");
read_buffer_iterator.setSchemaToLastFile(ColumnsDescription(names_and_types));
schemas_for_union_mode.emplace_back(names_and_types, read_buffer_iterator.getLastFileName());
}
if (format_name && mode == SchemaInferenceMode::DEFAULT)
break;
}
}
if (!format_name)
throw Exception(ErrorCodes::CANNOT_DETECT_FORMAT, "The data format cannot be detected by the contents of the files. You can specify the format manually");
/// If we got all schemas from cache, schema_reader can be uninitialized.
/// But we still need some stateless methods of ISchemaReader,
/// let's initialize it with empty buffer.
EmptyReadBuffer empty;
if (!schema_reader)
schema_reader = FormatFactory::instance().getSchemaReader(format_name, empty, context, format_settings);
schema_reader = FormatFactory::instance().getSchemaReader(*format_name, empty, context, format_settings);
if (mode == SchemaInferenceMode::UNION)
{
@ -273,11 +467,23 @@ try
}
if (names_and_types.empty())
{
if (iterations <= 1)
{
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file. "
"Error: {}. You can specify the structure manually",
*format_name,
exception_messages);
}
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"All attempts to extract table structure from files failed. "
"Errors:{}\nYou can specify the structure manually",
"Errors:\n{}\nYou can specify the structure manually",
exception_messages);
}
/// If we have "INSERT SELECT" query then try to order
/// columns as they are ordered in table schema for formats
@ -294,22 +500,22 @@ try
if (ordered_list)
names_and_types = *ordered_list;
}
/// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure.
names_and_types.erase(
std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }),
names_and_types.end());
auto columns = ColumnsDescription(names_and_types);
if (mode == SchemaInferenceMode::DEFAULT)
read_buffer_iterator.setResultingSchema(columns);
return {columns, *format_name};
}
else
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"{} file format doesn't support schema inference. You must specify the structure manually",
format_name);
/// Some formats like CSVWithNames can contain empty column names. We don't support empty column names and further processing can fail with an exception. Let's just remove columns with empty names from the structure.
names_and_types.erase(
std::remove_if(names_and_types.begin(), names_and_types.end(), [](const NameAndTypePair & pair) { return pair.name.empty(); }),
names_and_types.end());
auto columns = ColumnsDescription(names_and_types);
if (mode == SchemaInferenceMode::DEFAULT)
read_buffer_iterator.setResultingSchema(columns);
return columns;
throw Exception(
ErrorCodes::BAD_ARGUMENTS,
"{} file format doesn't support schema inference. You must specify the structure manually",
*format_name);
}
catch (Exception & e)
{
@ -319,16 +525,21 @@ catch (Exception & e)
throw;
}
ColumnsDescription readSchemaFromFormat(
const String & format_name,
const std::optional<FormatSettings> & format_settings,
IReadBufferIterator & read_buffer_iterator,
bool retry,
ContextPtr & context)
const ContextPtr & context)
{
std::unique_ptr<ReadBuffer> buf_out;
return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out);
return readSchemaFromFormatImpl(format_name, format_settings, read_buffer_iterator, context).first;
}
std::pair<ColumnsDescription, String> detectFormatAndReadSchema(
const std::optional<FormatSettings> & format_settings,
IReadBufferIterator & read_buffer_iterator,
const ContextPtr & context)
{
return readSchemaFromFormatImpl(std::nullopt, format_settings, read_buffer_iterator, context);
}
SchemaCache::Key getKeyForSchemaCache(

View File

@ -7,29 +7,68 @@
namespace DB
{
namespace ErrorCodes
{
extern const int NOT_IMPLEMENTED;
}
struct IReadBufferIterator
{
virtual ~IReadBufferIterator() = default;
virtual void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> /* buffer */) {}
/// Return read buffer of the next file or cached schema.
/// In DEFAULT schema inference mode cached schema can be from any file.
/// In UNION mode cached schema can be only from current file.
/// When there is no files to process, return pair (nullptr, nullopt)
virtual std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() = 0;
struct Data
{
/// Read buffer of the next file. Can be nullptr if there are no more files
/// or when schema was found in cache.
std::unique_ptr<ReadBuffer> buf;
/// Schema from cache.
/// In DEFAULT schema inference mode cached schema can be from any file.
/// In UNION mode cached schema can be only from current file.
std::optional<ColumnsDescription> cached_columns;
/// Format of the file if known.
std::optional<String> format_name;
};
virtual Data next() = 0;
/// Set read buffer returned in previous iteration.
virtual void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> /* buffer */) {}
/// Set number of rows to last file extracted during schema inference.
/// Used for caching number of rows from files metadata during schema inference.
virtual void setNumRowsToLastFile(size_t /*num_rows*/) {}
/// Set schema inferred from last file. Used for UNION mode to cache schema
/// per file.
virtual void setSchemaToLastFile(const ColumnsDescription & /*columns*/) {}
/// Set resulting inferred schema. Used for DEFAULT mode to cache schema
/// for all files.
virtual void setResultingSchema(const ColumnsDescription & /*columns*/) {}
/// Set auto detected format name.
virtual void setFormatName(const String & /*format_name*/) {}
/// Get last processed file name for better exception messages.
virtual String getLastFileName() const { return ""; }
/// Return true if method recreateLastReadBuffer is implemented.
virtual bool supportsLastReadBufferRecreation() const { return false; }
/// Recreate last read buffer to read data from the same file again.
/// Used to detect format from the file content to avoid
/// copying data.
virtual std::unique_ptr<ReadBuffer> recreateLastReadBuffer()
{
throw Exception(ErrorCodes::NOT_IMPLEMENTED, "Method recreateLastReadBuffer is not implemented");
}
};
struct SingleReadBufferIterator : public IReadBufferIterator
@ -39,12 +78,22 @@ public:
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
if (done)
return {nullptr, {}};
return {nullptr, {}, std::nullopt};
done = true;
return {std::move(buf), {}};
return Data{std::move(buf), {}, std::nullopt};
}
void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> buf_) override
{
buf = std::move(buf_);
}
std::unique_ptr<ReadBuffer> releaseBuffer()
{
return std::move(buf);
}
private:
@ -73,17 +122,16 @@ ColumnsDescription readSchemaFromFormat(
const String & format_name,
const std::optional<FormatSettings> & format_settings,
IReadBufferIterator & read_buffer_iterator,
bool retry,
ContextPtr & context);
const ContextPtr & context);
/// If ReadBuffer is created, it will be written to buf_out.
ColumnsDescription readSchemaFromFormat(
const String & format_name,
/// Try to detect the format of the data and it's schema.
/// It runs schema inference for some set of formats on the same file.
/// If schema reader of some format successfully inferred the schema from
/// some file, we consider that the data is in this format.
std::pair<ColumnsDescription, String> detectFormatAndReadSchema(
const std::optional<FormatSettings> & format_settings,
IReadBufferIterator & read_buffer_iterator,
bool retry,
ContextPtr & context,
std::unique_ptr<ReadBuffer> & buf_out);
const ContextPtr & context);
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);
SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context);

View File

@ -56,6 +56,7 @@ public:
/// It's possible to convert a file enumerator to a read buffer and vice versa.
virtual std::unique_ptr<ReadBufferFromFileBase> readFile(std::unique_ptr<FileEnumerator> enumerator) = 0;
virtual std::unique_ptr<FileEnumerator> nextFile(std::unique_ptr<ReadBuffer> read_buffer) = 0;
virtual std::unique_ptr<FileEnumerator> currentFile(std::unique_ptr<ReadBuffer> read_buffer) = 0;
virtual std::vector<std::string> getAllFiles() = 0;
virtual std::vector<std::string> getAllFiles(NameFilter filter) = 0;

View File

@ -340,6 +340,15 @@ std::unique_ptr<LibArchiveReader::FileEnumerator> LibArchiveReader::nextFile(std
return std::make_unique<FileEnumeratorImpl>(std::move(handle));
}
std::unique_ptr<LibArchiveReader::FileEnumerator> LibArchiveReader::currentFile(std::unique_ptr<ReadBuffer> read_buffer)
{
if (!dynamic_cast<ReadBufferFromLibArchive *>(read_buffer.get()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()");
auto read_buffer_from_libarchive = std::unique_ptr<ReadBufferFromLibArchive>(static_cast<ReadBufferFromLibArchive *>(read_buffer.release()));
auto handle = std::move(*read_buffer_from_libarchive).releaseHandle();
return std::make_unique<FileEnumeratorImpl>(std::move(handle));
}
std::vector<std::string> LibArchiveReader::getAllFiles()
{
return getAllFiles({});

View File

@ -40,6 +40,7 @@ public:
/// It's possible to convert a file enumerator to a read buffer and vice versa.
std::unique_ptr<ReadBufferFromFileBase> readFile(std::unique_ptr<FileEnumerator> enumerator) override;
std::unique_ptr<FileEnumerator> nextFile(std::unique_ptr<ReadBuffer> read_buffer) override;
std::unique_ptr<FileEnumerator> currentFile(std::unique_ptr<ReadBuffer> read_buffer) override;
std::vector<std::string> getAllFiles() override;
std::vector<std::string> getAllFiles(NameFilter filter) override;

View File

@ -589,6 +589,15 @@ std::unique_ptr<ZipArchiveReader::FileEnumerator> ZipArchiveReader::nextFile(std
return std::make_unique<FileEnumeratorImpl>(std::move(handle));
}
std::unique_ptr<ZipArchiveReader::FileEnumerator> ZipArchiveReader::currentFile(std::unique_ptr<ReadBuffer> read_buffer)
{
if (!dynamic_cast<ReadBufferFromZipArchive *>(read_buffer.get()))
throw Exception(ErrorCodes::LOGICAL_ERROR, "Wrong ReadBuffer passed to nextFile()");
auto read_buffer_from_zip = std::unique_ptr<ReadBufferFromZipArchive>(static_cast<ReadBufferFromZipArchive *>(read_buffer.release()));
auto handle = std::move(*read_buffer_from_zip).releaseHandle();
return std::make_unique<FileEnumeratorImpl>(std::move(handle));
}
std::vector<std::string> ZipArchiveReader::getAllFiles()
{
return getAllFiles({});

View File

@ -47,6 +47,7 @@ public:
/// It's possible to convert a file enumerator to a read buffer and vice versa.
std::unique_ptr<ReadBufferFromFileBase> readFile(std::unique_ptr<FileEnumerator> enumerator) override;
std::unique_ptr<FileEnumerator> nextFile(std::unique_ptr<ReadBuffer> read_buffer) override;
std::unique_ptr<FileEnumerator> currentFile(std::unique_ptr<ReadBuffer> read_buffer) override;
std::vector<std::string> getAllFiles() override;
std::vector<std::string> getAllFiles(NameFilter filter) override;

View File

@ -81,7 +81,7 @@ IIRowSchemaReader::IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & fo
{
}
void IIRowSchemaReader::setContext(ContextPtr & context)
void IIRowSchemaReader::setContext(const ContextPtr & context)
{
ColumnsDescription columns;
if (tryParseColumnsListFromString(hints_str, columns, context, hints_parsing_error))

View File

@ -34,7 +34,7 @@ public:
virtual bool hasStrictOrderOfColumns() const { return true; }
virtual bool needContext() const { return false; }
virtual void setContext(ContextPtr &) {}
virtual void setContext(const ContextPtr &) {}
virtual void setMaxRowsAndBytesToRead(size_t, size_t) {}
virtual size_t getNumRowsRead() const { return 0; }
@ -56,7 +56,7 @@ public:
IIRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, DataTypePtr default_type_ = nullptr);
bool needContext() const override { return !hints_str.empty(); }
void setContext(ContextPtr & context) override;
void setContext(const ContextPtr & context) override;
protected:
void setMaxRowsAndBytesToRead(size_t max_rows, size_t max_bytes) override

View File

@ -215,7 +215,7 @@ JSONColumnsSchemaReaderBase::JSONColumnsSchemaReaderBase(
{
}
void JSONColumnsSchemaReaderBase::setContext(ContextPtr & ctx)
void JSONColumnsSchemaReaderBase::setContext(const ContextPtr & ctx)
{
ColumnsDescription columns;
if (tryParseColumnsListFromString(hints_str, columns, ctx, hints_parsing_error))

View File

@ -84,7 +84,7 @@ public:
void transformTypesFromDifferentFilesIfNeeded(DataTypePtr & type, DataTypePtr & new_type) override;
bool needContext() const override { return !hints_str.empty(); }
void setContext(ContextPtr & ctx) override;
void setContext(const ContextPtr & ctx) override;
void setMaxRowsAndBytesToRead(size_t max_rows, size_t max_bytes) override
{

View File

@ -70,27 +70,36 @@ void JSONRowInputFormat::resetReadBuffer()
JSONEachRowRowInputFormat::resetReadBuffer();
}
JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_)
: JSONRowSchemaReader(std::make_unique<PeekableReadBuffer>(in_), format_settings_)
JSONRowSchemaReader::JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, bool fallback_to_json_each_row_)
: JSONRowSchemaReader(std::make_unique<PeekableReadBuffer>(in_), format_settings_, fallback_to_json_each_row_)
{
}
JSONRowSchemaReader::JSONRowSchemaReader(std::unique_ptr<PeekableReadBuffer> buf, const DB::FormatSettings & format_settings_)
: JSONEachRowSchemaReader(*buf, format_settings_), peekable_buf(std::move(buf))
JSONRowSchemaReader::JSONRowSchemaReader(std::unique_ptr<PeekableReadBuffer> buf, const DB::FormatSettings & format_settings_, bool fallback_to_json_each_row_)
: JSONEachRowSchemaReader(*buf, format_settings_), peekable_buf(std::move(buf)), fallback_to_json_each_row(fallback_to_json_each_row_)
{
}
NamesAndTypesList JSONRowSchemaReader::readSchema()
{
skipBOMIfExists(*peekable_buf);
PeekableReadBufferCheckpoint checkpoint(*peekable_buf);
/// Try to parse metadata, if failed, try to parse data as JSONEachRow format
NamesAndTypesList names_and_types;
if (JSONUtils::checkAndSkipObjectStart(*peekable_buf) && JSONUtils::tryReadMetadata(*peekable_buf, names_and_types))
return names_and_types;
peekable_buf->rollbackToCheckpoint(true);
return JSONEachRowSchemaReader::readSchema();
if (fallback_to_json_each_row)
{
PeekableReadBufferCheckpoint checkpoint(*peekable_buf);
/// Try to parse metadata, if failed, try to parse data as JSONEachRow format
NamesAndTypesList names_and_types;
if (JSONUtils::checkAndSkipObjectStart(*peekable_buf) && JSONUtils::tryReadMetadata(*peekable_buf, names_and_types))
return names_and_types;
peekable_buf->rollbackToCheckpoint(true);
return JSONEachRowSchemaReader::readSchema();
}
else
{
JSONUtils::skipObjectStart(*peekable_buf);
return JSONUtils::readMetadata(*peekable_buf);
}
}
void registerInputFormatJSON(FormatFactory & factory)
@ -109,19 +118,19 @@ void registerInputFormatJSON(FormatFactory & factory)
void registerJSONSchemaReader(FormatFactory & factory)
{
auto register_schema_reader = [&](const String & format)
auto register_schema_reader = [&](const String & format, bool fallback_to_json_each_row)
{
factory.registerSchemaReader(
format, [](ReadBuffer & buf, const FormatSettings & format_settings) { return std::make_unique<JSONRowSchemaReader>(buf, format_settings); });
format, [fallback_to_json_each_row](ReadBuffer & buf, const FormatSettings & format_settings) { return std::make_unique<JSONRowSchemaReader>(buf, format_settings, fallback_to_json_each_row); });
factory.registerAdditionalInfoForSchemaCacheGetter(format, [](const FormatSettings & settings)
{
return getAdditionalFormatInfoByEscapingRule(settings, FormatSettings::EscapingRule::JSON);
});
};
register_schema_reader("JSON");
register_schema_reader("JSON", true);
/// JSONCompact has the same suffix with metadata.
register_schema_reader("JSONCompact");
register_schema_reader("JSONCompact", false);
}
}

View File

@ -45,16 +45,17 @@ private:
class JSONRowSchemaReader : public JSONEachRowSchemaReader
{
public:
JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_);
JSONRowSchemaReader(ReadBuffer & in_, const FormatSettings & format_settings_, bool fallback_to_json_each_row_);
NamesAndTypesList readSchema() override;
bool hasStrictOrderOfColumns() const override { return false; }
private:
JSONRowSchemaReader(std::unique_ptr<PeekableReadBuffer> buf, const FormatSettings & format_settings_);
JSONRowSchemaReader(std::unique_ptr<PeekableReadBuffer> buf, const FormatSettings & format_settings_, bool fallback_to_json_each_row_);
std::unique_ptr<PeekableReadBuffer> peekable_buf;
bool fallback_to_json_each_row;
};
}

View File

@ -609,7 +609,9 @@ void registerTemplateSchemaReader(FormatFactory & factory)
{
size_t index = 0;
auto idx_getter = [&](const String &) -> std::optional<size_t> { return index++; };
auto row_format = fillRowFormat(settings, idx_getter, false);
ParsedTemplateFormatString row_format;
if (!settings.template_settings.row_format.empty())
row_format = fillRowFormat(settings, idx_getter, false);
std::unordered_set<FormatSettings::EscapingRule> visited_escaping_rules;
String result = fmt::format("row_format={}, resultset_format={}, row_between_delimiter={}",
settings.template_settings.row_format,

View File

@ -37,7 +37,7 @@ public:
void resetReadBuffer() override;
/// TODO: remove context somehow.
void setContext(ContextPtr & context_) { context = Context::createCopy(context_); }
void setContext(const ContextPtr & context_) { context = Context::createCopy(context_); }
const BlockMissingValues & getMissingValues() const override { return block_missing_values; }

View File

@ -35,7 +35,6 @@
#include <Server/TCPServer.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/StorageS3Cluster.h>
#include <Core/ExternalTable.h>
#include <Core/ServerSettings.h>
#include <Access/AccessControl.h>

View File

@ -38,25 +38,25 @@ public:
static ColumnsDescription getTableStructureFromData(
Configuration & base_configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr local_context)
const ContextPtr & local_context)
{
auto configuration = getConfigurationForDataRead(base_configuration, local_context);
return Storage::getTableStructureFromData(configuration, format_settings, local_context);
}
static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context)
static Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context)
{
return Storage::getConfiguration(engine_args, local_context, /* get_format_from_file */false);
}
Configuration updateConfigurationAndGetCopy(ContextPtr local_context) override
Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context) override
{
std::lock_guard lock(configuration_update_mutex);
updateConfigurationImpl(local_context);
return Storage::getConfiguration();
}
void updateConfiguration(ContextPtr local_context) override
void updateConfiguration(const ContextPtr & local_context) override
{
std::lock_guard lock(configuration_update_mutex);
updateConfigurationImpl(local_context);
@ -64,7 +64,7 @@ public:
private:
static Configuration getConfigurationForDataRead(
const Configuration & base_configuration, ContextPtr local_context, const Strings & keys = {})
const Configuration & base_configuration, const ContextPtr & local_context, const Strings & keys = {})
{
auto configuration{base_configuration};
configuration.update(local_context);
@ -84,12 +84,12 @@ private:
return configuration;
}
static Strings getDataFiles(const Configuration & configuration, ContextPtr local_context)
static Strings getDataFiles(const Configuration & configuration, const ContextPtr & local_context)
{
return MetadataParser().getFiles(configuration, local_context);
}
void updateConfigurationImpl(ContextPtr local_context)
void updateConfigurationImpl(const ContextPtr & local_context)
{
const bool updated = base_configuration.update(local_context);
auto new_keys = getDataFiles(base_configuration, local_context);

View File

@ -40,7 +40,7 @@ StorageIceberg::StorageIceberg(
ColumnsDescription StorageIceberg::getTableStructureFromData(
Configuration & base_configuration,
const std::optional<FormatSettings> &,
ContextPtr local_context)
const ContextPtr & local_context)
{
auto configuration{base_configuration};
configuration.update(local_context);
@ -48,7 +48,7 @@ ColumnsDescription StorageIceberg::getTableStructureFromData(
return ColumnsDescription(metadata->getTableSchema());
}
void StorageIceberg::updateConfigurationImpl(ContextPtr local_context)
void StorageIceberg::updateConfigurationImpl(const ContextPtr & local_context)
{
const bool updated = base_configuration.update(local_context);
auto new_metadata = parseIcebergMetadata(base_configuration, local_context);

View File

@ -51,28 +51,28 @@ public:
static ColumnsDescription getTableStructureFromData(
Configuration & base_configuration,
const std::optional<FormatSettings> &,
ContextPtr local_context);
const ContextPtr & local_context);
static Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context)
{
return StorageS3::getConfiguration(engine_args, local_context, /* get_format_from_file */false);
}
Configuration updateConfigurationAndGetCopy(ContextPtr local_context) override
Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context) override
{
std::lock_guard lock(configuration_update_mutex);
updateConfigurationImpl(local_context);
return StorageS3::getConfiguration();
}
void updateConfiguration(ContextPtr local_context) override
void updateConfiguration(const ContextPtr & local_context) override
{
std::lock_guard lock(configuration_update_mutex);
updateConfigurationImpl(local_context);
}
private:
void updateConfigurationImpl(ContextPtr local_context);
void updateConfigurationImpl(const ContextPtr & local_context);
std::unique_ptr<IcebergMetadata> current_metadata;
Configuration base_configuration;

View File

@ -67,6 +67,7 @@ namespace ErrorCodes
extern const int BAD_ARGUMENTS;
extern const int LOGICAL_ERROR;
extern const int CANNOT_COMPILE_REGEXP;
extern const int CANNOT_DETECT_FORMAT;
}
namespace
{
@ -194,7 +195,7 @@ StorageHDFS::StorageHDFS(
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_,
const bool distributed_processing_,
ASTPtr partition_by_)
@ -206,7 +207,8 @@ StorageHDFS::StorageHDFS(
, distributed_processing(distributed_processing_)
, partition_by(partition_by_)
{
FormatFactory::instance().checkFormatName(format_name);
if (format_name != "auto")
FormatFactory::instance().checkFormatName(format_name);
context_->getRemoteHostFilter().checkURL(Poco::URI(uri_));
checkHDFSURL(uri_);
@ -217,11 +219,19 @@ StorageHDFS::StorageHDFS(
if (columns_.empty())
{
auto columns = getTableStructureFromData(format_name, uri_, compression_method, context_);
ColumnsDescription columns;
if (format_name == "auto")
std::tie(columns, format_name) = getTableStructureAndFormatFromData(uri_, compression_method_, context_);
else
columns = getTableStructureFromData(format_name, uri_, compression_method, context_);
storage_metadata.setColumns(columns);
}
else
{
if (format_name == "auto")
format_name = getTableStructureAndFormatFromData(uri_, compression_method_, context_).second;
/// We don't allow special columns in HDFS storage.
if (!columns_.hasOnlyOrdinary())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine HDFS doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL");
@ -243,25 +253,25 @@ namespace
ReadBufferIterator(
const std::vector<StorageHDFS::PathWithInfo> & paths_with_info_,
const String & uri_without_path_,
const String & format_,
std::optional<String> format_,
const String & compression_method_,
const ContextPtr & context_)
: WithContext(context_)
, paths_with_info(paths_with_info_)
, uri_without_path(uri_without_path_)
, format(format_)
, format(std::move(format_))
, compression_method(compression_method_)
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
bool is_first = current_index == 0;
/// For default mode check cached columns for all paths on first iteration.
if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
{
if (auto cached_columns = tryGetColumnsFromCache(paths_with_info))
return {nullptr, cached_columns};
return {nullptr, cached_columns, format};
}
StorageHDFS::PathWithInfo path_with_info;
@ -271,10 +281,17 @@ namespace
if (current_index == paths_with_info.size())
{
if (is_first)
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because all files are empty. "
"You must specify table structure manually", format);
return {nullptr, std::nullopt};
{
if (format)
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because all files are empty. "
"You can specify table structure manually", *format);
throw Exception(
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files, because all files are empty. You can specify table structure manually");
}
return {nullptr, std::nullopt, format};
}
path_with_info = paths_with_info[current_index++];
@ -285,7 +302,7 @@ namespace
{
std::vector<StorageHDFS::PathWithInfo> paths = {path_with_info};
if (auto cached_columns = tryGetColumnsFromCache(paths))
return {nullptr, cached_columns};
return {nullptr, cached_columns, format};
}
auto compression = chooseCompressionMethod(path_with_info.path, compression_method);
@ -293,7 +310,7 @@ namespace
if (!getContext()->getSettingsRef().hdfs_skip_empty_files || !impl->eof())
{
const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max;
return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast<int>(zstd_window_log_max)), std::nullopt};
return {wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast<int>(zstd_window_log_max)), std::nullopt, format};
}
}
}
@ -304,7 +321,7 @@ namespace
return;
String source = uri_without_path + paths_with_info[current_index - 1].path;
auto key = getKeyForSchemaCache(source, format, std::nullopt, getContext());
auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext());
StorageHDFS::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -315,7 +332,7 @@ namespace
return;
String source = uri_without_path + paths_with_info[current_index - 1].path;
auto key = getKeyForSchemaCache(source, format, std::nullopt, getContext());
auto key = getKeyForSchemaCache(source, *format, std::nullopt, getContext());
StorageHDFS::getSchemaCache(getContext()).addColumns(key, columns);
}
@ -328,10 +345,15 @@ namespace
Strings sources;
sources.reserve(paths_with_info.size());
std::transform(paths_with_info.begin(), paths_with_info.end(), std::back_inserter(sources), [&](const StorageHDFS::PathWithInfo & path_with_info){ return uri_without_path + path_with_info.path; });
auto cache_keys = getKeysForSchemaCache(sources, format, {}, getContext());
auto cache_keys = getKeysForSchemaCache(sources, *format, {}, getContext());
StorageHDFS::getSchemaCache(getContext()).addManyColumns(cache_keys, columns);
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
String getLastFileName() const override
{
if (current_index != 0)
@ -340,13 +362,27 @@ namespace
return "";
}
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
chassert(current_index > 0 && current_index <= paths_with_info.size());
auto path_with_info = paths_with_info[current_index - 1];
auto compression = chooseCompressionMethod(path_with_info.path, compression_method);
auto impl = std::make_unique<ReadBufferFromHDFS>(uri_without_path, path_with_info.path, getContext()->getGlobalContext()->getConfigRef(), getContext()->getReadSettings());
const Int64 zstd_window_log_max = getContext()->getSettingsRef().zstd_window_log_max;
return wrapReadBufferWithCompressionMethod(std::move(impl), compression, static_cast<int>(zstd_window_log_max));
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromCache(const std::vector<StorageHDFS::PathWithInfo> & paths_with_info_)
{
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_hdfs)
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_hdfs)
return std::nullopt;
auto & schema_cache = StorageHDFS::getSchemaCache(getContext());
auto & schema_cache = StorageHDFS::getSchemaCache(context);
for (const auto & path_with_info : paths_with_info_)
{
auto get_last_mod_time = [&]() -> std::optional<time_t>
@ -354,7 +390,7 @@ namespace
if (path_with_info.info)
return path_with_info.info->last_mod_time;
auto builder = createHDFSBuilder(uri_without_path + "/", getContext()->getGlobalContext()->getConfigRef());
auto builder = createHDFSBuilder(uri_without_path + "/", context->getGlobalContext()->getConfigRef());
auto fs = createHDFSFS(builder.get());
HDFSFileInfoPtr hdfs_info(hdfsGetPathInfo(fs.get(), path_with_info.path.c_str()));
if (hdfs_info)
@ -364,10 +400,28 @@ namespace
};
String url = uri_without_path + path_with_info.path;
auto cache_key = getKeyForSchemaCache(url, format, {}, getContext());
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(url, *format, {}, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(url, format_name, {}, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
/// Now format is known. It should be the same for all files.
format = format_name;
return columns;
}
}
}
}
return std::nullopt;
@ -375,29 +429,49 @@ namespace
const std::vector<StorageHDFS::PathWithInfo> & paths_with_info;
const String & uri_without_path;
const String & format;
std::optional<String> format;
const String & compression_method;
size_t current_index = 0;
};
}
ColumnsDescription StorageHDFS::getTableStructureFromData(
const String & format,
std::pair<ColumnsDescription, String> StorageHDFS::getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const String & uri,
const String & compression_method,
ContextPtr ctx)
const ContextPtr & ctx)
{
const auto [path_from_uri, uri_without_path] = getPathFromUriAndUriWithoutPath(uri);
auto paths_with_info = getPathsList(path_from_uri, uri, ctx);
if (paths_with_info.empty() && !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format))
if (paths_with_info.empty() && (!format || !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(*format)))
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because there are no files in HDFS with provided path."
" You can specify table structure manually", *format);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files in HDFS with provided path."
" You must specify table structure manually", format);
"The data format cannot be detected by the contents of the files, because there are no files in HDFS with provided path."
" You can specify the format manually");
}
ReadBufferIterator read_buffer_iterator(paths_with_info, uri_without_path, format, compression_method, ctx);
return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, paths_with_info.size() > 1, ctx);
if (format)
return {readSchemaFromFormat(*format, std::nullopt, read_buffer_iterator, ctx), *format};
return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, ctx);
}
std::pair<ColumnsDescription, String> StorageHDFS::getTableStructureAndFormatFromData(const String & uri, const String & compression_method, const ContextPtr & ctx)
{
return getTableStructureAndFormatFromDataImpl(std::nullopt, uri, compression_method, ctx);
}
ColumnsDescription StorageHDFS::getTableStructureFromData(const String & format, const String & uri, const String & compression_method, const DB::ContextPtr & ctx)
{
return getTableStructureAndFormatFromDataImpl(format, uri, compression_method, ctx).first;
}
class HDFSSource::DisclosedGlobIterator::Impl
@ -533,7 +607,7 @@ StorageHDFS::PathWithInfo HDFSSource::URISIterator::next()
HDFSSource::HDFSSource(
const ReadFromFormatInfo & info,
StorageHDFSPtr storage_,
ContextPtr context_,
const ContextPtr & context_,
UInt64 max_block_size_,
std::shared_ptr<IteratorWrapper> file_iterator_,
bool need_only_count_)
@ -712,7 +786,7 @@ public:
HDFSSink(const String & uri,
const String & format,
const Block & sample_block,
ContextPtr context,
const ContextPtr & context,
const CompressionMethod compression_method)
: SinkToStorage(sample_block)
{
@ -1073,7 +1147,7 @@ void registerStorageHDFS(StorageFactory & factory)
}
if (format_name == "auto")
format_name = FormatFactory::instance().getFormatFromFileName(url, true);
format_name = FormatFactory::instance().getFormatFromFileName(url);
String compression_method;
if (engine_args.size() == 3)

View File

@ -44,7 +44,7 @@ public:
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_ = "",
bool distributed_processing_ = false,
ASTPtr partition_by = nullptr);
@ -86,7 +86,12 @@ public:
const String & format,
const String & uri,
const String & compression_method,
ContextPtr ctx);
const ContextPtr & ctx);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromData(
const String & uri,
const String & compression_method,
const ContextPtr & ctx);
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
@ -97,6 +102,12 @@ protected:
friend class ReadFromHDFS;
private:
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const String & uri,
const String & compression_method,
const ContextPtr & ctx);
std::vector<String> uris;
String format_name;
String compression_method;
@ -141,7 +152,7 @@ public:
HDFSSource(
const ReadFromFormatInfo & info,
StorageHDFSPtr storage_,
ContextPtr context_,
const ContextPtr & context_,
UInt64 max_block_size_,
std::shared_ptr<IteratorWrapper> file_iterator_,
bool need_only_count_);

View File

@ -43,12 +43,10 @@ StorageHDFSCluster::StorageHDFSCluster(
const String & format_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & compression_method_,
bool structure_argument_was_provided_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_)
const String & compression_method)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageHDFSCluster (" + table_id_.table_name + ")"))
, uri(uri_)
, format_name(format_name_)
, compression_method(compression_method_)
{
checkHDFSURL(uri_);
context_->getRemoteHostFilter().checkURL(Poco::URI(uri_));
@ -57,11 +55,20 @@ StorageHDFSCluster::StorageHDFSCluster(
if (columns_.empty())
{
auto columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_);
ColumnsDescription columns;
if (format_name == "auto")
std::tie(columns, format_name) = StorageHDFS::getTableStructureAndFormatFromData(uri_, compression_method, context_);
else
columns = StorageHDFS::getTableStructureFromData(format_name, uri_, compression_method, context_);
storage_metadata.setColumns(columns);
}
else
{
if (format_name == "auto")
format_name = StorageHDFS::getTableStructureAndFormatFromData(uri_, compression_method, context_).second;
storage_metadata.setColumns(columns_);
}
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
@ -69,13 +76,14 @@ StorageHDFSCluster::StorageHDFSCluster(
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
}
void StorageHDFSCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
void StorageHDFSCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function hdfsCluster, got '{}'", queryToString(query));
TableFunctionHDFSCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
TableFunctionHDFSCluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context);
}

View File

@ -28,8 +28,7 @@ public:
const String & format_name_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & compression_method_,
bool structure_argument_was_provided_);
const String & compression_method);
std::string getName() const override { return "HDFSCluster"; }
@ -42,11 +41,10 @@ public:
bool supportsTrivialCountOptimization() const override { return true; }
private:
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
String uri;
String format_name;
String compression_method;
NamesAndTypesList virtual_columns;
};

View File

@ -32,12 +32,10 @@ namespace DB
IStorageCluster::IStorageCluster(
const String & cluster_name_,
const StorageID & table_id_,
Poco::Logger * log_,
bool structure_argument_was_provided_)
Poco::Logger * log_)
: IStorage(table_id_)
, log(log_)
, cluster_name(cluster_name_)
, structure_argument_was_provided(structure_argument_was_provided_)
{
}
@ -130,8 +128,7 @@ void IStorageCluster::read(
query_to_send = interpreter.getQueryInfo().query->clone();
}
if (!structure_argument_was_provided)
addColumnsStructureToQuery(query_to_send, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), context);
updateQueryToSendIfNeeded(query_to_send, storage_snapshot, context);
RestoreQualifiedNamesVisitor::Data data;
data.distributed_table = DatabaseAndTableWithAlias(*getTableExpression(query_info.query->as<ASTSelectQuery &>(), 0));

View File

@ -19,8 +19,7 @@ public:
IStorageCluster(
const String & cluster_name_,
const StorageID & table_id_,
Poco::Logger * log_,
bool structure_argument_was_provided_);
Poco::Logger * log_);
void read(
QueryPlan & query_plan,
@ -42,13 +41,11 @@ public:
protected:
virtual void updateBeforeRead(const ContextPtr &) {}
virtual void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) = 0;
virtual void updateQueryToSendIfNeeded(ASTPtr & /*query*/, const StorageSnapshotPtr & /*storage_snapshot*/, const ContextPtr & /*context*/) {}
private:
Poco::Logger * log;
String cluster_name;
bool structure_argument_was_provided;
};

View File

@ -143,11 +143,17 @@ StorageS3Queue::StorageS3Queue(
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = StorageS3::getTableStructureFromDataImpl(configuration, format_settings, context_);
ColumnsDescription columns;
if (configuration.format == "auto")
std::tie(columns, configuration.format) = StorageS3::getTableStructureAndFormatFromData(configuration, format_settings, context_);
else
columns = StorageS3::getTableStructureFromData(configuration, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
{
if (configuration.format == "auto")
configuration.format = StorageS3::getTableStructureAndFormatFromData(configuration, format_settings, context_).second;
storage_metadata.setColumns(columns_);
}

View File

@ -65,6 +65,7 @@ namespace ErrorCodes
extern const int DATABASE_ACCESS_DENIED;
extern const int CANNOT_COMPILE_REGEXP;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int CANNOT_DETECT_FORMAT;
extern const int LOGICAL_ERROR;
extern const int NOT_IMPLEMENTED;
@ -127,7 +128,7 @@ void StorageAzureBlob::processNamedCollectionResult(StorageAzureBlob::Configurat
}
StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, ContextPtr local_context)
StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine_args, const ContextPtr & local_context)
{
StorageAzureBlob::Configuration configuration;
@ -143,7 +144,7 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine
configuration.blobs_paths = {configuration.blob_path};
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true);
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path);
return configuration;
}
@ -236,13 +237,13 @@ StorageAzureBlob::Configuration StorageAzureBlob::getConfiguration(ASTs & engine
configuration.blobs_paths = {configuration.blob_path};
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true);
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path);
return configuration;
}
AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(ContextPtr local_context)
AzureObjectStorage::SettingsPtr StorageAzureBlob::createSettings(const ContextPtr & local_context)
{
const auto & context_settings = local_context->getSettingsRef();
auto settings_ptr = std::make_unique<AzureObjectStorageSettings>();
@ -447,7 +448,7 @@ Poco::URI StorageAzureBlob::Configuration::getConnectionURL() const
StorageAzureBlob::StorageAzureBlob(
const Configuration & configuration_,
std::unique_ptr<AzureObjectStorage> && object_storage_,
ContextPtr context,
const ContextPtr & context,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
@ -463,17 +464,25 @@ StorageAzureBlob::StorageAzureBlob(
, format_settings(format_settings_)
, partition_by(partition_by_)
{
FormatFactory::instance().checkFormatName(configuration.format);
if (configuration.format != "auto")
FormatFactory::instance().checkFormatName(configuration.format);
context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.getConnectionURL());
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context, distributed_processing);
ColumnsDescription columns;
if (configuration.format == "auto")
std::tie(columns, configuration.format) = getTableStructureAndFormatFromData(object_storage.get(), configuration, format_settings, context);
else
columns = getTableStructureFromData(object_storage.get(), configuration, format_settings, context);
storage_metadata.setColumns(columns);
}
else
{
if (configuration.format == "auto")
configuration.format = getTableStructureAndFormatFromData(object_storage.get(), configuration, format_settings, context).second;
/// We don't allow special columns in File storage.
if (!columns_.hasOnlyOrdinary())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine AzureBlobStorage doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL");
@ -517,7 +526,7 @@ public:
StorageAzureBlobSink(
const String & format,
const Block & sample_block_,
ContextPtr context,
const ContextPtr & context,
std::optional<FormatSettings> format_settings_,
const CompressionMethod compression_method,
AzureObjectStorage * object_storage,
@ -607,22 +616,21 @@ private:
std::mutex cancel_mutex;
};
class PartitionedStorageAzureBlobSink : public PartitionedSink
class PartitionedStorageAzureBlobSink : public PartitionedSink, WithContext
{
public:
PartitionedStorageAzureBlobSink(
const ASTPtr & partition_by,
const String & format_,
const Block & sample_block_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
const CompressionMethod compression_method_,
AzureObjectStorage * object_storage_,
const String & blob_)
: PartitionedSink(partition_by, context_, sample_block_)
: PartitionedSink(partition_by, context_, sample_block_), WithContext(context_)
, format(format_)
, sample_block(sample_block_)
, context(context_)
, compression_method(compression_method_)
, object_storage(object_storage_)
, blob(blob_)
@ -638,7 +646,7 @@ public:
return std::make_shared<StorageAzureBlobSink>(
format,
sample_block,
context,
getContext(),
format_settings,
compression_method,
object_storage,
@ -649,7 +657,6 @@ public:
private:
const String format;
const Block sample_block;
const ContextPtr context;
const CompressionMethod compression_method;
AzureObjectStorage * object_storage;
const String blob;
@ -913,7 +920,7 @@ StorageAzureBlobSource::GlobIterator::GlobIterator(
String blob_path_with_globs_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns_,
ContextPtr context_,
const ContextPtr & context_,
RelativePathsWithMetadata * outer_blobs_,
std::function<void(FileProgress)> file_progress_callback_)
: IIterator(context_)
@ -1028,7 +1035,7 @@ StorageAzureBlobSource::KeysIterator::KeysIterator(
const Strings & keys_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns_,
ContextPtr context_,
const ContextPtr & context_,
RelativePathsWithMetadata * outer_blobs,
std::function<void(FileProgress)> file_progress_callback)
: IIterator(context_)
@ -1147,7 +1154,7 @@ StorageAzureBlobSource::StorageAzureBlobSource(
const ReadFromFormatInfo & info,
const String & format_,
String name_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
UInt64 max_block_size_,
String compression_hint_,
@ -1290,6 +1297,7 @@ namespace
ReadBufferIterator(
const std::shared_ptr<StorageAzureBlobSource::IIterator> & file_iterator_,
AzureObjectStorage * object_storage_,
std::optional<String> format_,
const StorageAzureBlob::Configuration & configuration_,
const std::optional<FormatSettings> & format_settings_,
const RelativePathsWithMetadata & read_keys_,
@ -1298,19 +1306,20 @@ namespace
, file_iterator(file_iterator_)
, object_storage(object_storage_)
, configuration(configuration_)
, format(std::move(format_))
, format_settings(format_settings_)
, read_keys(read_keys_)
, prev_read_keys_size(read_keys_.size())
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
/// For default mode check cached columns for currently read keys on first iteration.
if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
{
if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end()))
return {nullptr, cached_columns};
return {nullptr, cached_columns, format};
}
current_path_with_metadata = file_iterator->next();
@ -1318,12 +1327,20 @@ namespace
if (current_path_with_metadata.relative_path.empty())
{
if (first)
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because there are no files with provided path "
"in AzureBlobStorage. You can specify table structure manually", *format);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path "
"in AzureBlobStorage. You must specify table structure manually", configuration.format);
"The data format cannot be detected by the contents of the files, because there are no files with provided path "
"in AzureBlobStorage. You can specify table structure manually");
}
return {nullptr, std::nullopt};
return {nullptr, std::nullopt, format};
}
first = false;
@ -1334,13 +1351,13 @@ namespace
auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end());
prev_read_keys_size = read_keys.size();
if (columns_from_cache)
return {nullptr, columns_from_cache};
return {nullptr, columns_from_cache, format};
}
else if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION)
{
RelativePathsWithMetadata paths = {current_path_with_metadata};
if (auto columns_from_cache = tryGetColumnsFromCache(paths.begin(), paths.end()))
return {nullptr, columns_from_cache};
return {nullptr, columns_from_cache, format};
}
first = false;
@ -1348,7 +1365,7 @@ namespace
return {wrapReadBufferWithCompressionMethod(
object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes),
chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method),
zstd_window_log_max), std::nullopt};
zstd_window_log_max), std::nullopt, format};
}
void setNumRowsToLastFile(size_t num_rows) override
@ -1357,7 +1374,7 @@ namespace
return;
String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path;
auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto key = getKeyForSchemaCache(source, *format, format_settings, getContext());
StorageAzureBlob::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -1368,7 +1385,7 @@ namespace
return;
String source = fs::path(configuration.connection_url) / configuration.container / current_path_with_metadata.relative_path;
auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto key = getKeyForSchemaCache(source, *format, format_settings, getContext());
StorageAzureBlob::getSchemaCache(getContext()).addColumns(key, columns);
}
@ -1382,16 +1399,36 @@ namespace
Strings sources;
sources.reserve(read_keys.size());
std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket + '/' + elem.relative_path; });
auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext());
auto cache_keys = getKeysForSchemaCache(sources, *format, format_settings, getContext());
StorageAzureBlob::getSchemaCache(getContext()).addManyColumns(cache_keys, columns);
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
String getLastFileName() const override { return current_path_with_metadata.relative_path; }
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
int zstd_window_log_max = static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max);
return wrapReadBufferWithCompressionMethod(
object_storage->readObject(StoredObject(current_path_with_metadata.relative_path), getContext()->getReadSettings(), {}, current_path_with_metadata.metadata.size_bytes),
chooseCompressionMethod(current_path_with_metadata.relative_path, configuration.compression_method),
zstd_window_log_max);
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromCache(const RelativePathsWithMetadata::const_iterator & begin, const RelativePathsWithMetadata::const_iterator & end)
{
auto & schema_cache = StorageAzureBlob::getSchemaCache(getContext());
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_azure)
return std::nullopt;
auto & schema_cache = StorageAzureBlob::getSchemaCache(context);
for (auto it = begin; it < end; ++it)
{
auto get_last_mod_time = [&] -> std::optional<time_t>
@ -1403,10 +1440,28 @@ namespace
auto host_and_bucket = configuration.connection_url + '/' + configuration.container;
String source = host_and_bucket + '/' + it->relative_path;
auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(source, *format, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(source, format_name, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
/// Now format is known. It should be the same for all files.
format = format_name;
return columns;
}
}
}
}
return std::nullopt;
@ -1415,6 +1470,7 @@ namespace
std::shared_ptr<StorageAzureBlobSource::IIterator> file_iterator;
AzureObjectStorage * object_storage;
const StorageAzureBlob::Configuration & configuration;
std::optional<String> format;
const std::optional<FormatSettings> & format_settings;
const RelativePathsWithMetadata & read_keys;
size_t prev_read_keys_size;
@ -1423,21 +1479,16 @@ namespace
};
}
ColumnsDescription StorageAzureBlob::getTableStructureFromData(
std::pair<ColumnsDescription, String> StorageAzureBlob::getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
AzureObjectStorage * object_storage,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx,
bool distributed_processing)
const ContextPtr & ctx)
{
RelativePathsWithMetadata read_keys;
std::shared_ptr<StorageAzureBlobSource::IIterator> file_iterator;
if (distributed_processing)
{
file_iterator = std::make_shared<StorageAzureBlobSource::ReadIterator>(ctx,
ctx->getReadTaskCallback());
}
else if (configuration.withGlobs())
if (configuration.withGlobs())
{
file_iterator = std::make_shared<StorageAzureBlobSource::GlobIterator>(
object_storage, configuration.container, configuration.blob_path, nullptr, NamesAndTypesList{}, ctx, &read_keys);
@ -1448,8 +1499,28 @@ ColumnsDescription StorageAzureBlob::getTableStructureFromData(
object_storage, configuration.container, configuration.blobs_paths, nullptr, NamesAndTypesList{}, ctx, &read_keys);
}
ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, configuration, format_settings, read_keys, ctx);
return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx);
ReadBufferIterator read_buffer_iterator(file_iterator, object_storage, format, configuration, format_settings, read_keys, ctx);
if (format)
return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, ctx), *format};
return detectFormatAndReadSchema(format_settings, read_buffer_iterator, ctx);
}
std::pair<ColumnsDescription, String> StorageAzureBlob::getTableStructureAndFormatFromData(
DB::AzureObjectStorage * object_storage,
const DB::StorageAzureBlob::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const DB::ContextPtr & ctx)
{
return getTableStructureAndFormatFromDataImpl(std::nullopt, object_storage, configuration, format_settings, ctx);
}
ColumnsDescription StorageAzureBlob::getTableStructureFromData(
DB::AzureObjectStorage * object_storage,
const DB::StorageAzureBlob::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const DB::ContextPtr & ctx)
{
return getTableStructureAndFormatFromDataImpl(std::nullopt, object_storage, configuration, format_settings, ctx).first;
}
SchemaCache & StorageAzureBlob::getSchemaCache(const ContextPtr & ctx)

View File

@ -31,9 +31,9 @@ public:
String getPath() const { return blob_path; }
bool update(ContextPtr context);
bool update(const ContextPtr & context);
void connect(ContextPtr context);
void connect(const ContextPtr & context);
bool withGlobs() const { return blob_path.find_first_of("*?{") != std::string::npos; }
@ -59,7 +59,7 @@ public:
StorageAzureBlob(
const Configuration & configuration_,
std::unique_ptr<AzureObjectStorage> && object_storage_,
ContextPtr context_,
const ContextPtr & context_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
@ -68,10 +68,10 @@ public:
bool distributed_processing_,
ASTPtr partition_by_);
static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context);
static StorageAzureBlob::Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context);
static AzureClientPtr createClient(StorageAzureBlob::Configuration configuration, bool is_read_only);
static AzureObjectStorage::SettingsPtr createSettings(ContextPtr local_context);
static AzureObjectStorage::SettingsPtr createSettings(const ContextPtr & local_context);
static void processNamedCollectionResult(StorageAzureBlob::Configuration & configuration, const NamedCollection & collection);
@ -115,10 +115,22 @@ public:
AzureObjectStorage * object_storage,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx,
bool distributed_processing = false);
const ContextPtr & ctx);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromData(
AzureObjectStorage * object_storage,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx);
private:
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
AzureObjectStorage * object_storage,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx);
friend class ReadFromAzureBlob;
std::string name;
@ -137,7 +149,7 @@ public:
class IIterator : public WithContext
{
public:
IIterator(ContextPtr context_):WithContext(context_) {}
IIterator(const ContextPtr & context_):WithContext(context_) {}
virtual ~IIterator() = default;
virtual RelativePathWithMetadata next() = 0;
@ -153,7 +165,7 @@ public:
String blob_path_with_globs_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns_,
ContextPtr context_,
const ContextPtr & context_,
RelativePathsWithMetadata * outer_blobs_,
std::function<void(FileProgress)> file_progress_callback_ = {});
@ -186,7 +198,7 @@ public:
class ReadIterator : public IIterator
{
public:
explicit ReadIterator(ContextPtr context_,
explicit ReadIterator(const ContextPtr & context_,
const ReadTaskCallback & callback_)
: IIterator(context_), callback(callback_) { }
RelativePathWithMetadata next() override
@ -207,7 +219,7 @@ public:
const Strings & keys_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns_,
ContextPtr context_,
const ContextPtr & context_,
RelativePathsWithMetadata * outer_blobs,
std::function<void(FileProgress)> file_progress_callback = {});
@ -229,7 +241,7 @@ public:
const ReadFromFormatInfo & info,
const String & format_,
String name_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
UInt64 max_block_size_,
String compression_hint_,

View File

@ -36,23 +36,30 @@ StorageAzureBlobCluster::StorageAzureBlobCluster(
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
bool structure_argument_was_provided_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageAzureBlobCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_)
const ContextPtr & context)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageAzureBlobCluster (" + table_id_.table_name + ")"))
, configuration{configuration_}
, object_storage(std::move(object_storage_))
{
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL());
context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.getConnectionURL());
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
ColumnsDescription columns;
/// `format_settings` is set to std::nullopt, because StorageAzureBlobCluster is used only as table function
auto columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context_, false);
if (configuration.format == "auto")
std::tie(columns, configuration.format) = StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context);
else
columns = StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context);
storage_metadata.setColumns(columns);
}
else
{
if (configuration.format == "auto")
configuration.format = StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, /*format_settings=*/std::nullopt, context).second;
storage_metadata.setColumns(columns_);
}
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
@ -60,13 +67,14 @@ StorageAzureBlobCluster::StorageAzureBlobCluster(
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
}
void StorageAzureBlobCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
void StorageAzureBlobCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query));
TableFunctionAzureBlobStorageCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
TableFunctionAzureBlobStorageCluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), configuration.format, context);
}
RemoteQueryExecutor::Extension StorageAzureBlobCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const

View File

@ -27,8 +27,7 @@ public:
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
bool structure_argument_was_provided_);
const ContextPtr & context);
std::string getName() const override { return "AzureBlobStorageCluster"; }
@ -43,7 +42,7 @@ public:
private:
void updateBeforeRead(const ContextPtr & /*context*/) override {}
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
StorageAzureBlob::Configuration configuration;
NamesAndTypesList virtual_columns;

View File

@ -89,6 +89,7 @@ namespace ErrorCodes
extern const int LOGICAL_ERROR;
extern const int CANNOT_APPEND_TO_FILE;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int CANNOT_DETECT_FORMAT;
extern const int CANNOT_COMPILE_REGEXP;
}
@ -327,7 +328,7 @@ std::unique_ptr<ReadBuffer> createReadBuffer(
}
Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read)
Strings StorageFile::getPathsList(const String & table_path, const String & user_files_path, const ContextPtr & context, size_t & total_bytes_to_read)
{
fs::path user_files_absolute_path = fs::weakly_canonical(user_files_path);
fs::path fs_table_path(table_path);
@ -374,27 +375,44 @@ namespace
public:
ReadBufferFromFileIterator(
const std::vector<String> & paths_,
const String & format_,
std::optional<String> format_,
const String & compression_method_,
const std::optional<FormatSettings> & format_settings_,
ContextPtr context_)
const ContextPtr & context_)
: WithContext(context_)
, paths(paths_)
, format(format_)
, format(std::move(format_))
, compression_method(compression_method_)
, format_settings(format_settings_)
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
bool is_first = current_index == 0;
/// For default mode check cached columns for all paths on first iteration.
/// If we have cached columns, next() won't be called again.
if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
if (is_first)
{
if (auto cached_columns = tryGetColumnsFromCache(paths))
return {nullptr, cached_columns};
/// If format is unknown we iterate through all paths on first iteration and
/// try to determine format by file name.
if (!format)
{
for (const auto & path : paths)
{
if (auto format_from_path = FormatFactory::instance().tryGetFormatFromFileName(path))
{
format = format_from_path;
break;
}
}
}
/// For default mode check cached columns for all paths on first iteration.
/// If we have cached columns, next() won't be called again.
if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
{
if (auto cached_columns = tryGetColumnsFromCache(paths))
return {nullptr, cached_columns, format};
}
}
String path;
@ -405,11 +423,18 @@ namespace
if (current_index == paths.size())
{
if (is_first)
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because all files are empty. You can specify the format manually",
*format);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually",
format);
return {nullptr, std::nullopt};
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files, because all files are empty. You can specify table structure manually");
}
return {nullptr, std::nullopt, std::nullopt};
}
path = paths[current_index++];
@ -420,10 +445,10 @@ namespace
if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION)
{
if (auto cached_columns = tryGetColumnsFromCache({path}))
return {nullptr, cached_columns};
return {nullptr, cached_columns, format};
}
return {createReadBuffer(path, file_stat, false, -1, compression_method, getContext()), std::nullopt};
return {createReadBuffer(path, file_stat, false, -1, compression_method, getContext()), std::nullopt, format};
}
void setNumRowsToLastFile(size_t num_rows) override
@ -431,7 +456,7 @@ namespace
if (!getContext()->getSettingsRef().use_cache_for_count_from_files)
return;
auto key = getKeyForSchemaCache(paths[current_index - 1], format, format_settings, getContext());
auto key = getKeyForSchemaCache(paths[current_index - 1], *format, format_settings, getContext());
StorageFile::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -443,7 +468,7 @@ namespace
/// For union mode, schema can be different for different files, so we need to
/// cache last inferred schema only for last processed file.
auto cache_key = getKeyForSchemaCache(paths[current_index - 1], format, format_settings, getContext());
auto cache_key = getKeyForSchemaCache(paths[current_index - 1], *format, format_settings, getContext());
StorageFile::getSchemaCache(getContext()).addColumns(cache_key, columns);
}
@ -454,7 +479,7 @@ namespace
return;
/// For default mode we cache resulting schema for all paths.
auto cache_keys = getKeysForSchemaCache(paths, format, format_settings, getContext());
auto cache_keys = getKeysForSchemaCache(paths, *format, format_settings, getContext());
StorageFile::getSchemaCache(getContext()).addManyColumns(cache_keys, columns);
}
@ -465,14 +490,30 @@ namespace
return "";
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
chassert(current_index > 0 && current_index <= paths.size());
auto path = paths[current_index - 1];
auto file_stat = getFileStat(path, false, -1, "File");
return createReadBuffer(path, file_stat, false, -1, compression_method, getContext());
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromCache(const Strings & paths_)
{
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_file)
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_file)
return std::nullopt;
/// Check if the cache contains one of the paths.
auto & schema_cache = StorageFile::getSchemaCache(getContext());
auto & schema_cache = StorageFile::getSchemaCache(context);
struct stat file_stat{};
for (const auto & path : paths_)
{
@ -484,10 +525,28 @@ namespace
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(path, format, format_settings, getContext());
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(path, *format, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(path, format_name, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
/// Now format is known. It should be the same for all files.
format = format_name;
return columns;
}
}
}
}
return std::nullopt;
@ -496,7 +555,7 @@ namespace
const std::vector<String> & paths;
size_t current_index = 0;
String format;
std::optional<String> format;
String compression_method;
const std::optional<FormatSettings> & format_settings;
};
@ -506,17 +565,17 @@ namespace
public:
ReadBufferFromArchiveIterator(
const StorageFile::ArchiveInfo & archive_info_,
const String & format_,
std::optional<String> format_,
const std::optional<FormatSettings> & format_settings_,
ContextPtr context_)
const ContextPtr & context_)
: WithContext(context_)
, archive_info(archive_info_)
, format(format_)
, format(std::move(format_))
, format_settings(format_settings_)
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
/// For default mode check cached columns for all initial archive paths (maybe with globs) on first iteration.
/// If we have cached columns, next() won't be called again.
@ -524,8 +583,8 @@ namespace
{
for (const auto & archive : archive_info.paths_to_archives)
{
if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, archive_info.path_in_archive))
return {nullptr, cached_columns};
if (auto cached_schema = tryGetSchemaFromCache(archive, fmt::format("{}::{}", archive, archive_info.path_in_archive)))
return {nullptr, cached_schema, format};
}
}
@ -535,12 +594,19 @@ namespace
if (current_archive_index == archive_info.paths_to_archives.size())
{
if (is_first)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because all files are empty. You must specify table structure manually",
format);
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because all files are empty. You can specify table structure manually",
*format);
return {nullptr, std::nullopt};
throw Exception(
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files, because all files are empty. You can specify the format manually");
}
return {nullptr, std::nullopt, format};
}
const auto & archive = archive_info.paths_to_archives[current_archive_index];
@ -554,11 +620,18 @@ namespace
continue;
}
if (format)
throw Exception(
ErrorCodes::CANNOT_DETECT_FORMAT,
"The table structure cannot be extracted from a {} format file, because the archive {} is empty. "
"You can specify table structure manually",
*format,
archive);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because the archive {} is empty. "
"You must specify table structure manually",
format,
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files, because the archive {} is empty. "
"You can specify the format manually",
archive);
}
@ -574,8 +647,8 @@ namespace
last_read_file_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), archive_info.path_in_archive));
is_first = false;
if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, last_read_file_path))
return {nullptr, cached_columns};
if (auto cached_schema = tryGetSchemaFromCache(archive, last_read_file_path))
return {nullptr, cached_schema, format};
}
else
{
@ -611,13 +684,20 @@ namespace
last_read_file_path = paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive_reader->getPath(), *filename));
is_first = false;
if (auto cached_columns = tryGetColumnsFromSchemaCache(archive, last_read_file_path))
/// If format is unknown we can try to determine it by the file name.
if (!format)
{
if (auto format_from_file = FormatFactory::instance().tryGetFormatFromFileName(*filename))
format = format_from_file;
}
if (auto cached_schema = tryGetSchemaFromCache(archive, last_read_file_path))
{
/// For union mode next() will be called again even if we found cached columns,
/// so we need to remember last_read_buffer to continue iterating through files in archive.
if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION)
last_read_buffer = archive_reader->readFile(std::move(file_enumerator));
return {nullptr, cached_columns};
return {nullptr, cached_schema, format};
}
read_buf = archive_reader->readFile(std::move(file_enumerator));
@ -626,7 +706,7 @@ namespace
break;
}
return {std::move(read_buf), std::nullopt};
return {std::move(read_buf), std::nullopt, format};
}
void setPreviousReadBuffer(std::unique_ptr<ReadBuffer> buffer) override
@ -640,7 +720,7 @@ namespace
if (!getContext()->getSettingsRef().use_cache_for_count_from_files)
return;
auto key = getKeyForSchemaCache(last_read_file_path, format, format_settings, getContext());
auto key = getKeyForSchemaCache(last_read_file_path, *format, format_settings, getContext());
StorageFile::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -653,7 +733,7 @@ namespace
/// For union mode, schema can be different for different files in archive, so we need to
/// cache last inferred schema only for last processed file.
auto & schema_cache = StorageFile::getSchemaCache(getContext());
auto cache_key = getKeyForSchemaCache(last_read_file_path, format, format_settings, getContext());
auto cache_key = getKeyForSchemaCache(last_read_file_path, *format, format_settings, getContext());
schema_cache.addColumns(cache_key, columns);
}
@ -669,17 +749,42 @@ namespace
for (const auto & archive : archive_info.paths_to_archives)
paths_for_schema_cache.emplace_back(fmt::format("{}::{}", archive, archive_info.path_in_archive));
auto & schema_cache = StorageFile::getSchemaCache(getContext());
auto cache_keys = getKeysForSchemaCache(paths_for_schema_cache, format, format_settings, getContext());
auto cache_keys = getKeysForSchemaCache(paths_for_schema_cache, *format, format_settings, getContext());
schema_cache.addManyColumns(cache_keys, columns);
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
String getLastFileName() const override
{
return last_read_file_path;
}
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
if (archive_info.isSingleFileRead())
{
chassert(current_archive_index > 0 && current_archive_index <= archive_info.paths_to_archives.size());
const auto & archive = archive_info.paths_to_archives[current_archive_index - 1];
auto archive_reader = createArchiveReader(archive);
return archive_reader->readFile(archive_info.path_in_archive, false);
}
chassert(current_archive_index >= 0 && current_archive_index < archive_info.paths_to_archives.size());
const auto & archive = archive_info.paths_to_archives[current_archive_index];
auto archive_reader = createArchiveReader(archive);
chassert(last_read_buffer);
file_enumerator = archive_reader->currentFile(std::move(last_read_buffer));
return archive_reader->readFile(std::move(file_enumerator));
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromSchemaCache(const std::string & archive_path, const std::string & full_path)
std::optional<ColumnsDescription> tryGetSchemaFromCache(const std::string & archive_path, const std::string & full_path)
{
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_file)
@ -695,11 +800,28 @@ namespace
return file_stat.st_mtime;
};
auto cache_key = getKeyForSchemaCache(full_path, format, format_settings, context);
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(full_path, *format, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(full_path, format_name, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
/// Now format is known. It should be the same for all files.
format = format_name;
return columns;
}
}
}
return std::nullopt;
}
@ -715,13 +837,13 @@ namespace
std::unique_ptr<IArchiveReader::FileEnumerator> file_enumerator;
std::unique_ptr<ReadBuffer> last_read_buffer;
String format;
std::optional<String> format;
const std::optional<FormatSettings> & format_settings;
std::vector<std::string> paths_for_schema_cache;
};
}
ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr context)
std::pair<ColumnsDescription, String> StorageFile::getTableStructureAndFormatFromFileDescriptor(std::optional<String> format, const ContextPtr & context)
{
/// If we want to read schema from file descriptor we should create
/// a read buffer from fd, create a checkpoint, read some data required
@ -738,22 +860,29 @@ ColumnsDescription StorageFile::getTableStructureFromFileDescriptor(ContextPtr c
read_buf->setCheckpoint();
auto read_buffer_iterator = SingleReadBufferIterator(std::move(read_buf));
auto columns = readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, false, context, peekable_read_buffer_from_fd);
ColumnsDescription columns;
if (format)
columns = readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context);
else
std::tie(columns, format) = detectFormatAndReadSchema(format_settings, read_buffer_iterator, context);
peekable_read_buffer_from_fd = read_buffer_iterator.releaseBuffer();
if (peekable_read_buffer_from_fd)
{
/// If we have created read buffer in readSchemaFromFormat we should rollback to checkpoint.
assert_cast<PeekableReadBuffer *>(peekable_read_buffer_from_fd.get())->rollbackToCheckpoint();
has_peekable_read_buffer_from_fd = true;
}
return columns;
return {columns, *format};
}
ColumnsDescription StorageFile::getTableStructureFromFile(
const String & format,
std::pair<ColumnsDescription, String> StorageFile::getTableStructureAndFormatFromFileImpl(
std::optional<String> format,
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
ContextPtr context,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info)
{
if (format == "Distributed")
@ -761,29 +890,60 @@ ColumnsDescription StorageFile::getTableStructureFromFile(
if (paths.empty())
throw Exception(ErrorCodes::INCORRECT_FILE_NAME, "Cannot get table structure from file, because no files match specified name");
return ColumnsDescription(DistributedAsyncInsertSource(paths[0]).getOutputs().front().getHeader().getNamesAndTypesList());
return {ColumnsDescription(DistributedAsyncInsertSource(paths[0]).getOutputs().front().getHeader().getNamesAndTypesList()), *format};
}
if (((archive_info && archive_info->paths_to_archives.empty()) || (!archive_info && paths.empty()))
&& !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(format))
&& (!format || !FormatFactory::instance().checkIfFormatHasExternalSchemaReader(*format)))
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because there are no files with provided path. "
"You can specify table structure manually", *format);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path. "
"You must specify table structure manually", format);
"The data format cannot be detected by the contents of the files, because there are no files with provided path. "
"You can specify the format manually");
}
if (archive_info)
{
ReadBufferFromArchiveIterator read_buffer_iterator(*archive_info, format, format_settings, context);
return readSchemaFromFormat(
format,
format_settings,
read_buffer_iterator,
/*retry=*/archive_info->paths_to_archives.size() > 1 || !archive_info->isSingleFileRead(),
context);
if (format)
return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format};
return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context);
}
ReadBufferFromFileIterator read_buffer_iterator(paths, format, compression_method, format_settings, context);
return readSchemaFromFormat(format, format_settings, read_buffer_iterator, paths.size() > 1, context);
if (format)
return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format};
return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context);
}
ColumnsDescription StorageFile::getTableStructureFromFile(
const DB::String & format,
const std::vector<String> & paths,
const DB::String & compression_method,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info)
{
return getTableStructureAndFormatFromFileImpl(format, paths, compression_method, format_settings, context, archive_info).first;
}
std::pair<ColumnsDescription, String> StorageFile::getTableStructureAndFormatFromFile(
const std::vector<String> & paths,
const DB::String & compression_method,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info)
{
return getTableStructureAndFormatFromFileImpl(std::nullopt, paths, compression_method, format_settings, context, archive_info);
}
bool StorageFile::supportsSubsetOfColumns(const ContextPtr & context) const
@ -874,7 +1034,7 @@ StorageFile::StorageFile(CommonArguments args)
, compression_method(args.compression_method)
, base_path(args.getContext()->getPath())
{
if (format_name != "Distributed")
if (format_name != "Distributed" && format_name != "auto")
FormatFactory::instance().checkFormatName(format_name);
}
@ -886,16 +1046,19 @@ void StorageFile::setStorageMetadata(CommonArguments args)
{
ColumnsDescription columns;
if (use_table_fd)
columns = getTableStructureFromFileDescriptor(args.getContext());
{
if (format_name == "auto")
std::tie(columns, format_name) = getTableStructureAndFormatFromFileDescriptor(std::nullopt, args.getContext());
else
columns = getTableStructureAndFormatFromFileDescriptor(format_name, args.getContext()).first;
}
else
{
columns = getTableStructureFromFile(
format_name,
paths,
compression_method,
format_settings,
args.getContext(),
archive_info);
if (format_name == "auto")
std::tie(columns, format_name) = getTableStructureAndFormatFromFile(paths, compression_method, format_settings, args.getContext(), archive_info);
else
columns = getTableStructureFromFile(format_name, paths, compression_method, format_settings, args.getContext(), archive_info);
if (!args.columns.empty() && args.columns != columns)
throw Exception(ErrorCodes::INCOMPATIBLE_COLUMNS, "Table structure and file structure are different");
}
@ -903,6 +1066,8 @@ void StorageFile::setStorageMetadata(CommonArguments args)
}
else
{
if (format_name == "auto")
format_name = getTableStructureAndFormatFromFile(paths, compression_method, format_settings, args.getContext(), archive_info).second;
/// We don't allow special columns in File storage.
if (!args.columns.hasOnlyOrdinary())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine File doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL");
@ -917,7 +1082,7 @@ void StorageFile::setStorageMetadata(CommonArguments args)
}
static std::chrono::seconds getLockTimeout(ContextPtr context)
static std::chrono::seconds getLockTimeout(const ContextPtr & context)
{
const Settings & settings = context->getSettingsRef();
Int64 lock_timeout = settings.lock_acquire_timeout.totalSeconds();
@ -933,9 +1098,9 @@ StorageFileSource::FilesIterator::FilesIterator(
std::optional<StorageFile::ArchiveInfo> archive_info_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns,
ContextPtr context_,
const ContextPtr & context_,
bool distributed_processing_)
: files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_), context(context_)
: WithContext(context_), files(files_), archive_info(std::move(archive_info_)), distributed_processing(distributed_processing_)
{
ActionsDAGPtr filter_dag;
if (!distributed_processing && !archive_info && !files.empty())
@ -948,7 +1113,7 @@ StorageFileSource::FilesIterator::FilesIterator(
String StorageFileSource::FilesIterator::next()
{
if (distributed_processing)
return context->getReadTaskCallback()();
return getContext()->getReadTaskCallback()();
else
{
const auto & fs = isReadFromArchive() ? archive_info->paths_to_archives : files;
@ -972,12 +1137,12 @@ const String & StorageFileSource::FilesIterator::getFileNameInArchive()
StorageFileSource::StorageFileSource(
const ReadFromFormatInfo & info,
std::shared_ptr<StorageFile> storage_,
ContextPtr context_,
const ContextPtr & context_,
UInt64 max_block_size_,
FilesIteratorPtr files_iterator_,
std::unique_ptr<ReadBuffer> read_buf_,
bool need_only_count_)
: SourceWithKeyCondition(info.source_header, false)
: SourceWithKeyCondition(info.source_header, false), WithContext(context_)
, storage(std::move(storage_))
, files_iterator(std::move(files_iterator_))
, read_buf(std::move(read_buf_))
@ -985,13 +1150,12 @@ StorageFileSource::StorageFileSource(
, requested_columns(info.requested_columns)
, requested_virtual_columns(info.requested_virtual_columns)
, block_for_format(info.format_header)
, context(context_)
, max_block_size(max_block_size_)
, need_only_count(need_only_count_)
{
if (!storage->use_table_fd)
{
shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(context));
shared_lock = std::shared_lock(storage->rwlock, getLockTimeout(getContext()));
if (!shared_lock)
throw Exception(ErrorCodes::TIMEOUT_EXCEEDED, "Lock timeout exceeded");
storage->readers_counter.fetch_add(1, std::memory_order_release);
@ -1008,7 +1172,7 @@ void StorageFileSource::beforeDestroy()
if (std::uncaught_exceptions() == 0 && cnt == 1 && !storage->was_renamed)
{
shared_lock.unlock();
auto exclusive_lock = std::unique_lock{storage->rwlock, getLockTimeout(context)};
auto exclusive_lock = std::unique_lock{storage->rwlock, getLockTimeout(getContext())};
if (!exclusive_lock)
return;
@ -1027,7 +1191,7 @@ void StorageFileSource::beforeDestroy()
file_path = file_path.lexically_normal();
// Checking access rights
checkCreationIsAllowed(context, context->getUserFilesPath(), file_path, true);
checkCreationIsAllowed(getContext(), getContext()->getUserFilesPath(), file_path, true);
// Checking an existing of new file
if (fs::exists(file_path))
@ -1060,7 +1224,7 @@ void StorageFileSource::setKeyCondition(const ActionsDAG::NodeRawConstPtrs & nod
bool StorageFileSource::tryGetCountFromCache(const struct stat & file_stat)
{
if (!context->getSettingsRef().use_cache_for_count_from_files)
if (!getContext()->getSettingsRef().use_cache_for_count_from_files)
return false;
auto num_rows_from_cache = tryGetNumRowsFromCache(current_path, file_stat.st_mtime);
@ -1102,7 +1266,7 @@ Chunk StorageFileSource::generate()
return {};
auto file_stat = getFileStat(archive, storage->use_table_fd, storage->table_fd, storage->getName());
if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0)
if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0)
continue;
archive_reader = createArchiveReader(archive);
@ -1116,7 +1280,7 @@ Chunk StorageFileSource::generate()
if (!read_buf)
continue;
if (auto progress_callback = context->getFileProgressCallback())
if (auto progress_callback = getContext()->getFileProgressCallback())
progress_callback(FileProgress(0, tryGetFileSizeFromReadBuffer(*read_buf).value_or(0)));
}
else
@ -1130,7 +1294,7 @@ Chunk StorageFileSource::generate()
return {};
current_archive_stat = getFileStat(archive, storage->use_table_fd, storage->table_fd, storage->getName());
if (context->getSettingsRef().engine_file_skip_empty_files && current_archive_stat.st_size == 0)
if (getContext()->getSettingsRef().engine_file_skip_empty_files && current_archive_stat.st_size == 0)
continue;
archive_reader = createArchiveReader(archive);
@ -1164,7 +1328,7 @@ Chunk StorageFileSource::generate()
continue;
read_buf = archive_reader->readFile(std::move(file_enumerator));
if (auto progress_callback = context->getFileProgressCallback())
if (auto progress_callback = getContext()->getFileProgressCallback())
progress_callback(FileProgress(0, tryGetFileSizeFromReadBuffer(*read_buf).value_or(0)));
}
}
@ -1190,16 +1354,16 @@ Chunk StorageFileSource::generate()
file_stat = getFileStat(current_path, storage->use_table_fd, storage->table_fd, storage->getName());
current_file_size = file_stat.st_size;
if (context->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0)
if (getContext()->getSettingsRef().engine_file_skip_empty_files && file_stat.st_size == 0)
continue;
if (need_only_count && tryGetCountFromCache(file_stat))
continue;
read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, context);
read_buf = createReadBuffer(current_path, file_stat, storage->use_table_fd, storage->table_fd, storage->compression_method, getContext());
}
const Settings & settings = context->getSettingsRef();
const Settings & settings = getContext()->getSettingsRef();
size_t file_num = 0;
if (storage->archive_info)
@ -1211,7 +1375,7 @@ Chunk StorageFileSource::generate()
const auto max_parsing_threads = std::max<size_t>(settings.max_threads / file_num, 1UL);
input_format = FormatFactory::instance().getInput(
storage->format_name, *read_buf, block_for_format, context, max_block_size, storage->format_settings,
storage->format_name, *read_buf, block_for_format, getContext(), max_block_size, storage->format_settings,
max_parsing_threads, std::nullopt, /*is_remote_fs*/ false, CompressionMethod::None, need_only_count);
if (key_condition)
@ -1227,7 +1391,7 @@ Chunk StorageFileSource::generate()
{
builder.addSimpleTransform([&](const Block & header)
{
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *input_format, context);
return std::make_shared<AddingDefaultsTransform>(header, columns_description, *input_format, getContext());
});
}
@ -1264,7 +1428,7 @@ Chunk StorageFileSource::generate()
if (storage->use_table_fd)
finished_generate = true;
if (input_format && storage->format_name != "Distributed" && context->getSettingsRef().use_cache_for_count_from_files)
if (input_format && storage->format_name != "Distributed" && getContext()->getSettingsRef().use_cache_for_count_from_files)
addNumRowsToCache(current_path, total_rows_in_file);
total_rows_in_file = 0;
@ -1295,14 +1459,14 @@ Chunk StorageFileSource::generate()
void StorageFileSource::addNumRowsToCache(const String & path, size_t num_rows) const
{
auto key = getKeyForSchemaCache(path, storage->format_name, storage->format_settings, context);
StorageFile::getSchemaCache(context).addNumRows(key, num_rows);
auto key = getKeyForSchemaCache(path, storage->format_name, storage->format_settings, getContext());
StorageFile::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
std::optional<size_t> StorageFileSource::tryGetNumRowsFromCache(const String & path, time_t last_mod_time) const
{
auto & schema_cache = StorageFile::getSchemaCache(context);
auto key = getKeyForSchemaCache(path, storage->format_name, storage->format_settings, context);
auto & schema_cache = StorageFile::getSchemaCache(getContext());
auto key = getKeyForSchemaCache(path, storage->format_name, storage->format_settings, getContext());
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
return last_mod_time;
@ -1311,7 +1475,7 @@ std::optional<size_t> StorageFileSource::tryGetNumRowsFromCache(const String & p
return schema_cache.tryGetNumRows(key, get_last_mod_time);
}
class ReadFromFile : public SourceStepWithFilter
class ReadFromFile : public SourceStepWithFilter, WithContext
{
public:
std::string getName() const override { return "ReadFromFile"; }
@ -1323,14 +1487,13 @@ public:
std::shared_ptr<StorageFile> storage_,
ReadFromFormatInfo info_,
const bool need_only_count_,
ContextPtr context_,
const ContextPtr & context_,
size_t max_block_size_,
size_t num_streams_)
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)})
: SourceStepWithFilter(DataStream{.header = std::move(sample_block)}), WithContext(context_)
, storage(std::move(storage_))
, info(std::move(info_))
, need_only_count(need_only_count_)
, context(std::move(context_))
, max_block_size(max_block_size_)
, max_num_streams(num_streams_)
{
@ -1341,7 +1504,6 @@ private:
ReadFromFormatInfo info;
const bool need_only_count;
ContextPtr context;
size_t max_block_size;
const size_t max_num_streams;
@ -1352,7 +1514,7 @@ private:
void ReadFromFile::applyFilters()
{
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, context);
auto filter_actions_dag = ActionsDAG::buildFilterActionsDAG(filter_nodes.nodes, {}, getContext());
const ActionsDAG::Node * predicate = nullptr;
if (filter_actions_dag)
predicate = filter_actions_dag->getOutputs().at(0);
@ -1422,7 +1584,7 @@ void ReadFromFile::createIterator(const ActionsDAG::Node * predicate)
storage->archive_info,
predicate,
storage->virtual_columns,
context,
getContext(),
storage->distributed_processing);
}
@ -1444,8 +1606,10 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui
Pipes pipes;
pipes.reserve(num_streams);
auto ctx = getContext();
/// Set total number of bytes to process. For progress bar.
auto progress_callback = context->getFileProgressCallback();
auto progress_callback = ctx->getFileProgressCallback();
if (progress_callback && !storage->archive_info)
progress_callback(FileProgress(0, storage->total_bytes_to_read));
@ -1463,20 +1627,20 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui
auto source = std::make_shared<StorageFileSource>(
info,
storage,
context,
ctx,
max_block_size,
files_iterator,
std::move(read_buffer),
need_only_count);
source->setKeyCondition(filter_nodes.nodes, context);
source->setKeyCondition(filter_nodes.nodes, ctx);
pipes.emplace_back(std::move(source));
}
auto pipe = Pipe::unitePipes(std::move(pipes));
size_t output_ports = pipe.numOutputPorts();
const bool parallelize_output = context->getSettingsRef().parallelize_output_from_storages;
if (parallelize_output && storage->parallelizeOutputAfterReading(context) && output_ports > 0 && output_ports < max_num_streams)
const bool parallelize_output = ctx->getSettingsRef().parallelize_output_from_storages;
if (parallelize_output && storage->parallelizeOutputAfterReading(ctx) && output_ports > 0 && output_ports < max_num_streams)
pipe.resize(max_num_streams);
if (pipe.empty())
@ -1489,7 +1653,7 @@ void ReadFromFile::initializePipeline(QueryPipelineBuilder & pipeline, const Bui
}
class StorageFileSink final : public SinkToStorage
class StorageFileSink final : public SinkToStorage, WithContext
{
public:
StorageFileSink(
@ -1502,9 +1666,9 @@ public:
const CompressionMethod compression_method_,
const std::optional<FormatSettings> & format_settings_,
const String format_name_,
ContextPtr context_,
const ContextPtr & context_,
int flags_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
: SinkToStorage(metadata_snapshot_->getSampleBlock()), WithContext(context_)
, metadata_snapshot(metadata_snapshot_)
, table_name_for_log(table_name_for_log_)
, table_fd(table_fd_)
@ -1514,7 +1678,6 @@ public:
, compression_method(compression_method_)
, format_name(format_name_)
, format_settings(format_settings_)
, context(context_)
, flags(flags_)
{
initialize();
@ -1531,9 +1694,9 @@ public:
const CompressionMethod compression_method_,
const std::optional<FormatSettings> & format_settings_,
const String format_name_,
ContextPtr context_,
const ContextPtr & context_,
int flags_)
: SinkToStorage(metadata_snapshot_->getSampleBlock())
: SinkToStorage(metadata_snapshot_->getSampleBlock()), WithContext(context_)
, metadata_snapshot(metadata_snapshot_)
, table_name_for_log(table_name_for_log_)
, table_fd(table_fd_)
@ -1543,7 +1706,6 @@ public:
, compression_method(compression_method_)
, format_name(format_name_)
, format_settings(format_settings_)
, context(context_)
, flags(flags_)
, lock(std::move(lock_))
{
@ -1567,7 +1729,7 @@ public:
/// In case of formats with prefixes if file is not empty we have already written prefix.
bool do_not_write_prefix = naked_buffer->size();
const auto & settings = context->getSettingsRef();
const auto & settings = getContext()->getSettingsRef();
write_buf = wrapWriteBufferWithCompressionMethod(
std::move(naked_buffer),
compression_method,
@ -1575,7 +1737,7 @@ public:
static_cast<int>(settings.output_format_compression_zstd_window_log));
writer = FormatFactory::instance().getOutputFormatParallelIfPossible(format_name,
*write_buf, metadata_snapshot->getSampleBlock(), context, format_settings);
*write_buf, metadata_snapshot->getSampleBlock(), getContext(), format_settings);
if (do_not_write_prefix)
writer->doNotWritePrefix();
@ -1658,7 +1820,6 @@ private:
std::string format_name;
std::optional<FormatSettings> format_settings;
ContextPtr context;
int flags;
std::unique_lock<std::shared_timed_mutex> lock;
@ -2043,7 +2204,7 @@ StorageFile::ArchiveInfo StorageFile::getArchiveInfo(
const std::string & path_to_archive,
const std::string & file_in_archive,
const std::string & user_files_path,
ContextPtr context,
const ContextPtr & context,
size_t & total_bytes_to_read
)
{

View File

@ -84,7 +84,7 @@ public:
static Names getVirtualColumnNames();
static Strings getPathsList(const String & table_path, const String & user_files_path, ContextPtr context, size_t & total_bytes_to_read);
static Strings getPathsList(const String & table_path, const String & user_files_path, const ContextPtr & context, size_t & total_bytes_to_read);
/// Check if the format supports reading only some subset of columns.
/// Is is useful because such formats could effectively skip unknown columns
@ -112,14 +112,19 @@ public:
}
};
ColumnsDescription getTableStructureFromFileDescriptor(ContextPtr context);
static ColumnsDescription getTableStructureFromFile(
const String & format,
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
ContextPtr context,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info = std::nullopt);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromFile(
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info = std::nullopt);
static SchemaCache & getSchemaCache(const ContextPtr & context);
@ -130,7 +135,7 @@ public:
const std::string & path_to_archive,
const std::string & file_in_archive,
const std::string & user_files_path,
ContextPtr context,
const ContextPtr & context,
size_t & total_bytes_to_read);
bool supportsTrivialCountOptimization() const override { return true; }
@ -141,6 +146,16 @@ protected:
friend class ReadFromFile;
private:
std::pair<ColumnsDescription, String> getTableStructureAndFormatFromFileDescriptor(std::optional<String> format, const ContextPtr & context);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromFileImpl(
std::optional<String> format,
const std::vector<String> & paths,
const String & compression_method,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context,
const std::optional<ArchiveInfo> & archive_info = std::nullopt);
void setStorageMetadata(CommonArguments args);
std::string format_name;
@ -187,10 +202,10 @@ private:
bool distributed_processing = false;
};
class StorageFileSource : public SourceWithKeyCondition
class StorageFileSource : public SourceWithKeyCondition, WithContext
{
public:
class FilesIterator
class FilesIterator : WithContext
{
public:
explicit FilesIterator(
@ -198,7 +213,7 @@ public:
std::optional<StorageFile::ArchiveInfo> archive_info_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns,
ContextPtr context_,
const ContextPtr & context_,
bool distributed_processing_ = false);
String next();
@ -227,8 +242,6 @@ private:
std::atomic<size_t> index = 0;
bool distributed_processing;
ContextPtr context;
};
using FilesIteratorPtr = std::shared_ptr<FilesIterator>;
@ -236,7 +249,7 @@ private:
StorageFileSource(
const ReadFromFormatInfo & info,
std::shared_ptr<StorageFile> storage_,
ContextPtr context_,
const ContextPtr & context_,
UInt64 max_block_size_,
FilesIteratorPtr files_iterator_,
std::unique_ptr<ReadBuffer> read_buf_,
@ -286,7 +299,6 @@ private:
NamesAndTypesList requested_virtual_columns;
Block block_for_format;
ContextPtr context; /// TODO Untangle potential issues with context lifetime.
UInt64 max_block_size;
bool finished_generate = false;

View File

@ -25,36 +25,39 @@ extern const int LOGICAL_ERROR;
}
StorageFileCluster::StorageFileCluster(
ContextPtr context_,
const ContextPtr & context,
const String & cluster_name_,
const String & filename_,
const String & format_name_,
const String & compression_method_,
const String & compression_method,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool structure_argument_was_provided_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageFileCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_)
const ConstraintsDescription & constraints_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageFileCluster (" + table_id_.table_name + ")"))
, filename(filename_)
, format_name(format_name_)
, compression_method(compression_method_)
{
StorageInMemoryMetadata storage_metadata;
size_t total_bytes_to_read; // its value isn't used as we are not reading files (just listing them). But it is required by getPathsList
paths = StorageFile::getPathsList(filename_, context_->getUserFilesPath(), context_, total_bytes_to_read);
paths = StorageFile::getPathsList(filename_, context->getUserFilesPath(), context, total_bytes_to_read);
if (columns_.empty())
{
auto columns = StorageFile::getTableStructureFromFile(format_name,
paths,
compression_method,
std::nullopt,
context_);
ColumnsDescription columns;
if (format_name == "auto")
std::tie(columns, format_name) = StorageFile::getTableStructureAndFormatFromFile(paths, compression_method, std::nullopt, context);
else
columns = StorageFile::getTableStructureFromFile(format_name, paths, compression_method, std::nullopt, context);
storage_metadata.setColumns(columns);
}
else
{
if (format_name == "auto")
format_name = StorageFile::getTableStructureAndFormatFromFile(paths, compression_method, std::nullopt, context).second;
storage_metadata.setColumns(columns_);
}
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
@ -62,13 +65,14 @@ StorageFileCluster::StorageFileCluster(
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
}
void StorageFileCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
void StorageFileCluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function fileCluster, got '{}'", queryToString(query));
TableFunctionFileCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
TableFunctionFileCluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context);
}
RemoteQueryExecutor::Extension StorageFileCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const

View File

@ -17,15 +17,14 @@ class StorageFileCluster : public IStorageCluster
{
public:
StorageFileCluster(
ContextPtr context_,
const ContextPtr & context_,
const String & cluster_name_,
const String & filename_,
const String & format_name_,
const String & compression_method_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
bool structure_argument_was_provided_);
const ConstraintsDescription & constraints_);
std::string getName() const override { return "FileCluster"; }
@ -38,12 +37,11 @@ public:
bool supportsTrivialCountOptimization() const override { return true; }
private:
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
Strings paths;
String filename;
String format_name;
String compression_method;
NamesAndTypesList virtual_columns;
};

View File

@ -129,6 +129,7 @@ namespace ErrorCodes
extern const int UNEXPECTED_EXPRESSION;
extern const int DATABASE_ACCESS_DENIED;
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
extern const int CANNOT_DETECT_FORMAT;
extern const int NOT_IMPLEMENTED;
extern const int CANNOT_COMPILE_REGEXP;
extern const int FILE_DOESNT_EXIST;
@ -428,7 +429,7 @@ StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(
const S3::URI & globbed_uri_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns_,
ContextPtr context,
const ContextPtr & context,
KeysWithInfo * read_keys_,
const S3Settings::RequestSettings & request_settings_,
std::function<void(FileProgress)> file_progress_callback_)
@ -563,7 +564,7 @@ StorageS3Source::StorageS3Source(
const ReadFromFormatInfo & info,
const String & format_,
String name_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
UInt64 max_block_size_,
const S3Settings::RequestSettings & request_settings_,
@ -841,7 +842,7 @@ public:
StorageS3Sink(
const String & format,
const Block & sample_block_,
ContextPtr context,
const ContextPtr & context,
std::optional<FormatSettings> format_settings_,
const CompressionMethod compression_method,
const StorageS3::Configuration & configuration_,
@ -949,23 +950,22 @@ private:
};
class PartitionedStorageS3Sink : public PartitionedSink
class PartitionedStorageS3Sink : public PartitionedSink, WithContext
{
public:
PartitionedStorageS3Sink(
const ASTPtr & partition_by,
const String & format_,
const Block & sample_block_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
const CompressionMethod compression_method_,
const StorageS3::Configuration & configuration_,
const String & bucket_,
const String & key_)
: PartitionedSink(partition_by, context_, sample_block_)
: PartitionedSink(partition_by, context_, sample_block_), WithContext(context_)
, format(format_)
, sample_block(sample_block_)
, context(context_)
, compression_method(compression_method_)
, configuration(configuration_)
, bucket(bucket_)
@ -985,7 +985,7 @@ public:
return std::make_shared<StorageS3Sink>(
format,
sample_block,
context,
getContext(),
format_settings,
compression_method,
configuration,
@ -997,7 +997,6 @@ public:
private:
const String format;
const Block sample_block;
const ContextPtr context;
const CompressionMethod compression_method;
const StorageS3::Configuration configuration;
const String bucket;
@ -1033,7 +1032,7 @@ private:
StorageS3::StorageS3(
const Configuration & configuration_,
ContextPtr context_,
const ContextPtr & context_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
@ -1050,18 +1049,27 @@ StorageS3::StorageS3(
{
updateConfiguration(context_); // NOLINT(clang-analyzer-optin.cplusplus.VirtualCall)
FormatFactory::instance().checkFormatName(configuration.format);
if (configuration.format != "auto")
FormatFactory::instance().checkFormatName(configuration.format);
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration.url.uri);
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration.headers_from_ast);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromDataImpl(configuration, format_settings, context_);
ColumnsDescription columns;
if (configuration.format == "auto")
std::tie(columns, configuration.format) = getTableStructureAndFormatFromData(configuration, format_settings, context_);
else
columns = getTableStructureFromData(configuration, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
{
if (configuration.format == "auto")
configuration.format = getTableStructureAndFormatFromData(configuration, format_settings, context_).second;
/// We don't allow special columns in S3 storage.
if (!columns_.hasOnlyOrdinary())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine S3 doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL");
@ -1350,14 +1358,14 @@ void StorageS3::truncate(const ASTPtr & /* query */, const StorageMetadataPtr &,
LOG_WARNING(&Poco::Logger::get("StorageS3"), "Failed to delete {}, error: {}", error.GetKey(), error.GetMessage());
}
StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(ContextPtr local_context)
StorageS3::Configuration StorageS3::updateConfigurationAndGetCopy(const ContextPtr & local_context)
{
std::lock_guard lock(configuration_update_mutex);
configuration.update(local_context);
return configuration;
}
void StorageS3::updateConfiguration(ContextPtr local_context)
void StorageS3::updateConfiguration(const ContextPtr & local_context)
{
std::lock_guard lock(configuration_update_mutex);
configuration.update(local_context);
@ -1375,7 +1383,7 @@ const StorageS3::Configuration & StorageS3::getConfiguration()
return configuration;
}
bool StorageS3::Configuration::update(ContextPtr context)
bool StorageS3::Configuration::update(const ContextPtr & context)
{
auto s3_settings = context->getStorageS3Settings().getSettings(url.uri.toString());
request_settings = s3_settings.request_settings;
@ -1390,7 +1398,7 @@ bool StorageS3::Configuration::update(ContextPtr context)
return true;
}
void StorageS3::Configuration::connect(ContextPtr context)
void StorageS3::Configuration::connect(const ContextPtr & context)
{
const Settings & global_settings = context->getGlobalContext()->getSettingsRef();
const Settings & local_settings = context->getSettingsRef();
@ -1462,7 +1470,7 @@ void StorageS3::processNamedCollectionResult(StorageS3::Configuration & configur
configuration.request_settings = S3Settings::RequestSettings(collection);
}
StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file)
StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file)
{
StorageS3::Configuration configuration;
@ -1601,7 +1609,7 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
configuration.keys = {configuration.url.key};
if (configuration.format == "auto" && get_format_from_file)
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.url.key, true);
configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.url.key).value_or("auto");
return configuration;
}
@ -1609,9 +1617,17 @@ StorageS3::Configuration StorageS3::getConfiguration(ASTs & engine_args, Context
ColumnsDescription StorageS3::getTableStructureFromData(
const StorageS3::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
const ContextPtr & ctx)
{
return getTableStructureFromDataImpl(configuration, format_settings, ctx);
return getTableStructureAndFormatFromDataImpl(configuration.format, configuration, format_settings, ctx).first;
}
std::pair<ColumnsDescription, String> StorageS3::getTableStructureAndFormatFromData(
const StorageS3::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx)
{
return getTableStructureAndFormatFromDataImpl(std::nullopt, configuration, format_settings, ctx);
}
namespace
@ -1623,24 +1639,43 @@ namespace
std::shared_ptr<StorageS3Source::IIterator> file_iterator_,
const StorageS3Source::KeysWithInfo & read_keys_,
const StorageS3::Configuration & configuration_,
std::optional<String> format_,
const std::optional<FormatSettings> & format_settings_,
const ContextPtr & context_)
: WithContext(context_)
, file_iterator(file_iterator_)
, read_keys(read_keys_)
, configuration(configuration_)
, format(std::move(format_))
, format_settings(format_settings_)
, prev_read_keys_size(read_keys_.size())
{
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
/// For default mode check cached columns for currently read keys on first iteration.
if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
if (first)
{
if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end()))
return {nullptr, cached_columns};
/// If format is unknown we iterate through all currently read keys on first iteration and
/// try to determine format by file name.
if (!format)
{
for (const auto & key_with_info : read_keys)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(key_with_info->key))
{
format = format_from_file_name;
break;
}
}
}
/// For default mode check cached columns for currently read keys on first iteration.
if (first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
{
if (auto cached_columns = tryGetColumnsFromCache(read_keys.begin(), read_keys.end()))
return {nullptr, cached_columns, format};
}
}
while (true)
@ -1650,13 +1685,34 @@ namespace
if (!current_key_with_info || current_key_with_info->key.empty())
{
if (first)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because there are no files with provided path "
"in S3 or all files are empty. You must specify table structure manually",
configuration.format);
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because there are no files with provided path "
"in S3 or all files are empty. You can specify table structure manually",
*format);
return {nullptr, std::nullopt};
throw Exception(
ErrorCodes::CANNOT_DETECT_FORMAT,
"The data format cannot be detected by the contents of the files, because there are no files with provided path "
"in S3 or all files are empty. You can specify the format manually");
}
return {nullptr, std::nullopt, format};
}
/// S3 file iterator could get new keys after new iteration, if format is unknown we can try to determine it by new file names.
if (!format && read_keys.size() > prev_read_keys_size)
{
for (auto it = read_keys.begin() + prev_read_keys_size; it != read_keys.end(); ++it)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName((*it)->key))
{
format = format_from_file_name;
break;
}
}
}
/// S3 file iterator could get new keys after new iteration, check them in schema cache if schema inference mode is default.
@ -1665,9 +1721,11 @@ namespace
auto columns_from_cache = tryGetColumnsFromCache(read_keys.begin() + prev_read_keys_size, read_keys.end());
prev_read_keys_size = read_keys.size();
if (columns_from_cache)
return {nullptr, columns_from_cache};
return {nullptr, columns_from_cache, format};
}
prev_read_keys_size = read_keys.size();
if (getContext()->getSettingsRef().s3_skip_empty_files && current_key_with_info->info && current_key_with_info->info->size == 0)
continue;
@ -1678,7 +1736,7 @@ namespace
if (auto columns_from_cache = tryGetColumnsFromCache(keys.begin(), keys.end()))
{
first = false;
return {nullptr, columns_from_cache};
return {nullptr, columns_from_cache, format};
}
}
@ -1687,7 +1745,7 @@ namespace
if (!getContext()->getSettingsRef().s3_skip_empty_files || !impl->eof())
{
first = false;
return {wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max), std::nullopt};
return {wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max), std::nullopt, format};
}
}
}
@ -1698,7 +1756,7 @@ namespace
return;
String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key;
auto key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto key = getKeyForSchemaCache(source, *format, format_settings, getContext());
StorageS3::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -1709,7 +1767,7 @@ namespace
return;
String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / configuration.url.bucket / current_key_with_info->key;
auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto cache_key = getKeyForSchemaCache(source, *format, format_settings, getContext());
StorageS3::getSchemaCache(getContext()).addColumns(cache_key, columns);
}
@ -1723,10 +1781,15 @@ namespace
Strings sources;
sources.reserve(read_keys.size());
std::transform(read_keys.begin(), read_keys.end(), std::back_inserter(sources), [&](const auto & elem){ return host_and_bucket / elem->key; });
auto cache_keys = getKeysForSchemaCache(sources, configuration.format, format_settings, getContext());
auto cache_keys = getKeysForSchemaCache(sources, *format, format_settings, getContext());
StorageS3::getSchemaCache(getContext()).addManyColumns(cache_keys, columns);
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
String getLastFileName() const override
{
if (current_key_with_info)
@ -1734,15 +1797,26 @@ namespace
return "";
}
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
chassert(current_key_with_info);
int zstd_window_log_max = static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max);
auto impl = std::make_unique<ReadBufferFromS3>(configuration.client, configuration.url.bucket, current_key_with_info->key, configuration.url.version_id, configuration.request_settings, getContext()->getReadSettings());
return wrapReadBufferWithCompressionMethod(std::move(impl), chooseCompressionMethod(current_key_with_info->key, configuration.compression_method), zstd_window_log_max);
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromCache(
const StorageS3::KeysWithInfo::const_iterator & begin,
const StorageS3::KeysWithInfo::const_iterator & end)
{
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_s3)
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_s3)
return std::nullopt;
auto & schema_cache = StorageS3::getSchemaCache(getContext());
auto & schema_cache = StorageS3::getSchemaCache(context);
for (auto it = begin; it < end; ++it)
{
auto get_last_mod_time = [&]
@ -1773,10 +1847,29 @@ namespace
String path = fs::path(configuration.url.bucket) / (*it)->key;
String source = fs::path(configuration.url.uri.getHost() + std::to_string(configuration.url.uri.getPort())) / path;
auto cache_key = getKeyForSchemaCache(source, configuration.format, format_settings, getContext());
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(source, *format, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(source, format_name, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
/// Now format is known. It should be the same for all files.
format = format_name;
return columns;
}
}
}
}
return std::nullopt;
@ -1785,6 +1878,7 @@ namespace
std::shared_ptr<StorageS3Source::IIterator> file_iterator;
const StorageS3Source::KeysWithInfo & read_keys;
const StorageS3::Configuration & configuration;
std::optional<String> format;
const std::optional<FormatSettings> & format_settings;
StorageS3Source::KeyWithInfoPtr current_key_with_info;
size_t prev_read_keys_size;
@ -1793,17 +1887,20 @@ namespace
}
ColumnsDescription StorageS3::getTableStructureFromDataImpl(
std::pair<ColumnsDescription, String> StorageS3::getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx)
const ContextPtr & ctx)
{
KeysWithInfo read_keys;
auto file_iterator = createFileIterator(configuration, false, ctx, {}, {}, &read_keys);
ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format_settings, ctx);
return readSchemaFromFormat(configuration.format, format_settings, read_buffer_iterator, configuration.withGlobs(), ctx);
ReadBufferIterator read_buffer_iterator(file_iterator, read_keys, configuration, format, format_settings, ctx);
if (format)
return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, ctx), *format};
return detectFormatAndReadSchema(format_settings, read_buffer_iterator, ctx);
}
void registerStorageS3Impl(const String & name, StorageFactory & factory)

View File

@ -80,7 +80,7 @@ public:
const S3::URI & globbed_uri_,
const ActionsDAG::Node * predicate,
const NamesAndTypesList & virtual_columns,
ContextPtr context,
const ContextPtr & context,
KeysWithInfo * read_keys_ = nullptr,
const S3Settings::RequestSettings & request_settings_ = {},
std::function<void(FileProgress)> progress_callback_ = {});
@ -134,7 +134,7 @@ public:
const ReadFromFormatInfo & info,
const String & format,
String name_,
ContextPtr context_,
const ContextPtr & context_,
std::optional<FormatSettings> format_settings_,
UInt64 max_block_size_,
const S3Settings::RequestSettings & request_settings_,
@ -280,9 +280,9 @@ public:
String getPath() const { return url.key; }
bool update(ContextPtr context);
bool update(const ContextPtr & context);
void connect(ContextPtr context);
void connect(const ContextPtr & context);
bool withGlobs() const { return url.key.find_first_of("*?{") != std::string::npos; }
@ -308,7 +308,7 @@ public:
StorageS3(
const Configuration & configuration_,
ContextPtr context_,
const ContextPtr & context_,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
@ -345,21 +345,26 @@ public:
static SchemaCache & getSchemaCache(const ContextPtr & ctx);
static StorageS3::Configuration getConfiguration(ASTs & engine_args, ContextPtr local_context, bool get_format_from_file = true);
static StorageS3::Configuration getConfiguration(ASTs & engine_args, const ContextPtr & local_context, bool get_format_from_file = true);
static ColumnsDescription getTableStructureFromData(
const StorageS3::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
const ContextPtr & ctx);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromData(
const StorageS3::Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & ctx);
using KeysWithInfo = StorageS3Source::KeysWithInfo;
bool supportsTrivialCountOptimization() const override { return true; }
protected:
virtual Configuration updateConfigurationAndGetCopy(ContextPtr local_context);
virtual Configuration updateConfigurationAndGetCopy(const ContextPtr & local_context);
virtual void updateConfiguration(ContextPtr local_context);
virtual void updateConfiguration(const ContextPtr & local_context);
void useConfiguration(const Configuration & new_configuration);
@ -380,10 +385,11 @@ private:
std::optional<FormatSettings> format_settings;
ASTPtr partition_by;
static ColumnsDescription getTableStructureFromDataImpl(
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const Configuration & configuration,
const std::optional<FormatSettings> & format_settings,
ContextPtr ctx);
const ContextPtr & ctx);
bool supportsSubcolumns() const override { return true; }

View File

@ -38,25 +38,34 @@ StorageS3Cluster::StorageS3Cluster(
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
bool structure_argument_was_provided_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"), structure_argument_was_provided_)
const ContextPtr & context)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageS3Cluster (" + table_id_.table_name + ")"))
, s3_configuration{configuration_}
{
context_->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
context_->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast);
context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast);
StorageInMemoryMetadata storage_metadata;
updateConfigurationIfChanged(context_);
updateConfigurationIfChanged(context);
if (columns_.empty())
{
ColumnsDescription columns;
/// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function
auto columns = StorageS3::getTableStructureFromDataImpl(s3_configuration, /*format_settings=*/std::nullopt, context_);
if (s3_configuration.format == "auto")
std::tie(columns, s3_configuration.format) = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context);
else
columns = StorageS3::getTableStructureFromData(s3_configuration, /*format_settings=*/std::nullopt, context);
storage_metadata.setColumns(columns);
}
else
{
if (s3_configuration.format == "auto")
s3_configuration.format = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context).second;
storage_metadata.setColumns(columns_);
}
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
@ -64,13 +73,17 @@ StorageS3Cluster::StorageS3Cluster(
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
}
void StorageS3Cluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
void StorageS3Cluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query));
TableFunctionS3Cluster::addColumnsStructureToArguments(expression_list->children, structure, context);
TableFunctionS3Cluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children,
storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(),
s3_configuration.format,
context);
}
void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context)

View File

@ -27,8 +27,7 @@ public:
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
bool structure_argument_was_provided_);
const ContextPtr & context_);
std::string getName() const override { return "S3Cluster"; }
@ -46,7 +45,7 @@ protected:
private:
void updateBeforeRead(const ContextPtr & context) override { updateConfigurationIfChanged(context); }
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
StorageS3::Configuration s3_configuration;
NamesAndTypesList virtual_columns;

View File

@ -101,7 +101,7 @@ static ConnectionTimeouts getHTTPTimeouts(ContextPtr context)
IStorageURLBase::IStorageURLBase(
const String & uri_,
ContextPtr context_,
const ContextPtr & context_,
const StorageID & table_id_,
const String & format_name_,
const std::optional<FormatSettings> & format_settings_,
@ -123,16 +123,26 @@ IStorageURLBase::IStorageURLBase(
, partition_by(partition_by_)
, distributed_processing(distributed_processing_)
{
FormatFactory::instance().checkFormatName(format_name);
if (format_name != "auto")
FormatFactory::instance().checkFormatName(format_name);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = getTableStructureFromData(format_name, uri, compression_method, headers, format_settings, context_);
ColumnsDescription columns;
if (format_name == "auto")
std::tie(columns, format_name) = getTableStructureAndFormatFromData(uri, compression_method, headers, format_settings, context_);
else
columns = getTableStructureFromData(format_name, uri, compression_method, headers, format_settings, context_);
storage_metadata.setColumns(columns);
}
else
{
if (format_name == "auto")
format_name = getTableStructureAndFormatFromData(uri, compression_method, headers, format_settings, context_).second;
/// We don't allow special columns in URL storage.
if (!columns_.hasOnlyOrdinary())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "Table engine URL doesn't support special columns like MATERIALIZED, ALIAS or EPHEMERAL");
@ -257,7 +267,7 @@ StorageURLSource::StorageURLSource(
const String & format_,
const std::optional<FormatSettings> & format_settings_,
String name_,
ContextPtr context_,
const ContextPtr & context_,
UInt64 max_block_size,
const ConnectionTimeouts & timeouts,
CompressionMethod compression_method,
@ -525,7 +535,7 @@ StorageURLSink::StorageURLSink(
const String & format,
const std::optional<FormatSettings> & format_settings,
const Block & sample_block,
ContextPtr context,
const ContextPtr & context,
const ConnectionTimeouts & timeouts,
const CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
@ -668,7 +678,7 @@ std::vector<std::pair<std::string, std::string>> IStorageURLBase::getReadURIPara
const Names & /*column_names*/,
const StorageSnapshotPtr & /*storage_snapshot*/,
const SelectQueryInfo & /*query_info*/,
ContextPtr /*context*/,
const ContextPtr & /*context*/,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t /*max_block_size*/) const
{
@ -679,7 +689,7 @@ std::function<void(std::ostream &)> IStorageURLBase::getReadPOSTDataCallback(
const Names & /*column_names*/,
const ColumnsDescription & /* columns_description */,
const SelectQueryInfo & /*query_info*/,
ContextPtr /*context*/,
const ContextPtr & /*context*/,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t /*max_block_size*/) const
{
@ -693,28 +703,48 @@ namespace
public:
ReadBufferIterator(
const std::vector<String> & urls_to_check_,
const String & format_,
std::optional<String> format_,
const CompressionMethod & compression_method_,
const HTTPHeaderEntries & headers_,
const std::optional<FormatSettings> & format_settings_,
const ContextPtr & context_)
: WithContext(context_), format(format_), compression_method(compression_method_), headers(headers_), format_settings(format_settings_)
: WithContext(context_), format(std::move(format_)), compression_method(compression_method_), headers(headers_), format_settings(format_settings_)
{
url_options_to_check.reserve(urls_to_check_.size());
for (const auto & url : urls_to_check_)
url_options_to_check.push_back(getFailoverOptions(url, getContext()->getSettingsRef().glob_expansion_max_elements));
}
std::pair<std::unique_ptr<ReadBuffer>, std::optional<ColumnsDescription>> next() override
Data next() override
{
bool is_first = (current_index == 0);
/// For default mode check cached columns for all urls on first iteration.
if (is_first && getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
if (is_first)
{
for (const auto & options : url_options_to_check)
/// If format is unknown we iterate through all url options on first iteration and
/// try to determine format by file name.
if (!format)
{
if (auto cached_columns = tryGetColumnsFromCache(options))
return {nullptr, cached_columns};
for (const auto & options : url_options_to_check)
{
for (const auto & url : options)
{
if (auto format_from_file_name = FormatFactory::instance().tryGetFormatFromFileName(url))
{
format = format_from_file_name;
break;
}
}
}
}
/// For default mode check cached columns for all urls on first iteration.
if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::DEFAULT)
{
for (const auto & options : url_options_to_check)
{
if (auto cached_columns = tryGetColumnsFromCache(options))
return {nullptr, cached_columns, format};
}
}
}
@ -724,20 +754,30 @@ namespace
if (current_index == url_options_to_check.size())
{
if (is_first)
{
if (format)
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"The table structure cannot be extracted from a {} format file, because all files are empty. "
"You can specify table structure manually",
*format);
throw Exception(
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
"Cannot extract table structure from {} format file, because all files are empty. "
"You must specify table structure manually",
format);
return {nullptr, std::nullopt};
"The data format cannot be detected by the contents of the files, because there are no files with provided path "
"You can specify the format manually");
}
return {nullptr, std::nullopt, format};
}
if (getContext()->getSettingsRef().schema_inference_mode == SchemaInferenceMode::UNION)
{
if (auto cached_columns = tryGetColumnsFromCache(url_options_to_check[current_index]))
if (auto cached_schema = tryGetColumnsFromCache(url_options_to_check[current_index]))
{
++current_index;
return {nullptr, cached_columns};
return {nullptr, cached_schema, format};
}
}
@ -762,7 +802,7 @@ namespace
return {wrapReadBufferWithCompressionMethod(
std::move(uri_and_buf.second),
compression_method,
static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max)), std::nullopt};
static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max)), std::nullopt, format};
}
void setNumRowsToLastFile(size_t num_rows) override
@ -770,7 +810,7 @@ namespace
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url)
return;
auto key = getKeyForSchemaCache(current_url_option, format, format_settings, getContext());
auto key = getKeyForSchemaCache(current_url_option, *format, format_settings, getContext());
StorageURL::getSchemaCache(getContext()).addNumRows(key, num_rows);
}
@ -780,7 +820,7 @@ namespace
|| getContext()->getSettingsRef().schema_inference_mode != SchemaInferenceMode::UNION)
return;
auto key = getKeyForSchemaCache(current_url_option, format, format_settings, getContext());
auto key = getKeyForSchemaCache(current_url_option, *format, format_settings, getContext());
StorageURL::getSchemaCache(getContext()).addColumns(key, columns);
}
@ -792,17 +832,45 @@ namespace
for (const auto & options : url_options_to_check)
{
auto keys = getKeysForSchemaCache(options, format, format_settings, getContext());
auto keys = getKeysForSchemaCache(options, *format, format_settings, getContext());
StorageURL::getSchemaCache(getContext()).addManyColumns(keys, columns);
}
}
void setFormatName(const String & format_name) override
{
format = format_name;
}
String getLastFileName() const override { return current_url_option; }
bool supportsLastReadBufferRecreation() const override { return true; }
std::unique_ptr<ReadBuffer> recreateLastReadBuffer() override
{
chassert(current_index > 0 && current_index <= url_options_to_check.size());
auto first_option = url_options_to_check[current_index - 1].cbegin();
auto uri_and_buf = StorageURLSource::getFirstAvailableURIAndReadBuffer(
first_option,
url_options_to_check[current_index - 1].cend(),
getContext(),
{},
Poco::Net::HTTPRequest::HTTP_GET,
{},
getHTTPTimeouts(getContext()),
credentials,
headers,
false,
false);
return wrapReadBufferWithCompressionMethod(std::move(uri_and_buf.second), compression_method, static_cast<int>(getContext()->getSettingsRef().zstd_window_log_max));
}
private:
std::optional<ColumnsDescription> tryGetColumnsFromCache(const Strings & urls)
{
if (!getContext()->getSettingsRef().schema_inference_use_cache_for_url)
auto context = getContext();
if (!context->getSettingsRef().schema_inference_use_cache_for_url)
return std::nullopt;
auto & schema_cache = StorageURL::getSchemaCache(getContext());
@ -810,7 +878,7 @@ namespace
{
auto get_last_mod_time = [&]() -> std::optional<time_t>
{
auto last_mod_time = StorageURL::tryGetLastModificationTime(url, headers, credentials, getContext());
auto last_mod_time = StorageURL::tryGetLastModificationTime(url, headers, credentials, context);
/// Some URLs could not have Last-Modified header, in this case we cannot be sure that
/// data wasn't changed after adding it's schema to cache. Use schema from cache only if
/// special setting for this case is enabled.
@ -819,10 +887,27 @@ namespace
return last_mod_time;
};
auto cache_key = getKeyForSchemaCache(url, format, format_settings, getContext());
auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time);
if (columns)
return columns;
if (format)
{
auto cache_key = getKeyForSchemaCache(url, *format, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
return columns;
}
else
{
/// If format is unknown, we can iterate through all possible input formats
/// and check if we have an entry with this format and this file in schema cache.
/// If we have such entry for some format, we can use this format to read the file.
for (const auto & format_name : FormatFactory::instance().getAllInputFormats())
{
auto cache_key = getKeyForSchemaCache(url, format_name, format_settings, context);
if (auto columns = schema_cache.tryGetColumns(cache_key, get_last_mod_time))
{
format = format_name;
return columns;
}
}
}
}
return std::nullopt;
@ -831,7 +916,7 @@ namespace
std::vector<std::vector<String>> url_options_to_check;
size_t current_index = 0;
String current_url_option;
const String & format;
std::optional<String> format;
const CompressionMethod & compression_method;
const HTTPHeaderEntries & headers;
Poco::Net::HTTPBasicCredentials credentials;
@ -839,13 +924,13 @@ namespace
};
}
ColumnsDescription IStorageURLBase::getTableStructureFromData(
const String & format,
std::pair<ColumnsDescription, String> IStorageURLBase::getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const String & uri,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
ContextPtr context)
const ContextPtr & context)
{
context->getRemoteHostFilter().checkURL(Poco::URI(uri));
@ -858,7 +943,30 @@ ColumnsDescription IStorageURLBase::getTableStructureFromData(
urls_to_check = {uri};
ReadBufferIterator read_buffer_iterator(urls_to_check, format, compression_method, headers, format_settings, context);
return readSchemaFromFormat(format, format_settings, read_buffer_iterator, urls_to_check.size() > 1, context);
if (format)
return {readSchemaFromFormat(*format, format_settings, read_buffer_iterator, context), *format};
return detectFormatAndReadSchema(format_settings, read_buffer_iterator, context);
}
ColumnsDescription IStorageURLBase::getTableStructureFromData(
const String & format,
const String & uri,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context)
{
return getTableStructureAndFormatFromDataImpl(format, uri, compression_method, headers, format_settings, context).first;
}
std::pair<ColumnsDescription, String> IStorageURLBase::getTableStructureAndFormatFromData(
const String & uri,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context)
{
return getTableStructureAndFormatFromDataImpl(std::nullopt, uri, compression_method, headers, format_settings, context);
}
bool IStorageURLBase::supportsSubsetOfColumns(const ContextPtr & context) const
@ -1243,7 +1351,7 @@ StorageURL::StorageURL(
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_,
const HTTPHeaderEntries & headers_,
const String & http_method_,
@ -1276,7 +1384,7 @@ StorageURLWithFailover::StorageURLWithFailover(
const std::optional<FormatSettings> & format_settings_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_)
: StorageURL("", table_id_, format_name_, format_settings_, columns_, constraints_, String{}, context_, compression_method_)
{
@ -1325,7 +1433,7 @@ FormatSettings StorageURL::getFormatSettingsFromArgs(const StorageFactory::Argum
}
size_t StorageURL::evalArgsAndCollectHeaders(
ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context)
ASTs & url_function_args, HTTPHeaderEntries & header_entries, const ContextPtr & context)
{
ASTs::iterator headers_it = url_function_args.end();
@ -1409,7 +1517,7 @@ void StorageURL::processNamedCollectionResult(Configuration & configuration, con
configuration.structure = collection.getOrDefault<String>("structure", "auto");
}
StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, ContextPtr local_context)
StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, const ContextPtr & local_context)
{
StorageURL::Configuration configuration;
@ -1433,7 +1541,7 @@ StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, ContextPtr l
}
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(configuration.url).getPath(), true);
configuration.format = FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(configuration.url).getPath()).value_or("auto");
for (const auto & [header, value] : configuration.headers)
{

View File

@ -57,7 +57,15 @@ public:
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
ContextPtr context);
const ContextPtr & context);
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromData(
const String & uri,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context);
static SchemaCache & getSchemaCache(const ContextPtr & context);
@ -72,7 +80,7 @@ protected:
IStorageURLBase(
const String & uri_,
ContextPtr context_,
const ContextPtr & context_,
const StorageID & id_,
const String & format_name_,
const std::optional<FormatSettings> & format_settings_,
@ -106,7 +114,7 @@ protected:
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
const SelectQueryInfo & query_info,
ContextPtr context,
const ContextPtr & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size) const;
@ -114,7 +122,7 @@ protected:
const Names & column_names,
const ColumnsDescription & columns_description,
const SelectQueryInfo & query_info,
ContextPtr context,
const ContextPtr & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size) const;
@ -127,6 +135,14 @@ protected:
bool supportsTrivialCountOptimization() const override { return true; }
private:
static std::pair<ColumnsDescription, String> getTableStructureAndFormatFromDataImpl(
std::optional<String> format,
const String & uri,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers,
const std::optional<FormatSettings> & format_settings,
const ContextPtr & context);
virtual Block getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const = 0;
};
@ -160,7 +176,7 @@ public:
const String & format,
const std::optional<FormatSettings> & format_settings,
String name_,
ContextPtr context,
const ContextPtr & context,
UInt64 max_block_size,
const ConnectionTimeouts & timeouts,
CompressionMethod compression_method,
@ -231,7 +247,7 @@ public:
const String & format,
const std::optional<FormatSettings> & format_settings,
const Block & sample_block,
ContextPtr context,
const ContextPtr & context,
const ConnectionTimeouts & timeouts,
CompressionMethod compression_method,
const HTTPHeaderEntries & headers = {},
@ -263,7 +279,7 @@ public:
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const String & comment,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_,
const HTTPHeaderEntries & headers_ = {},
const String & method_ = "",
@ -292,12 +308,12 @@ public:
std::string addresses_expr;
};
static Configuration getConfiguration(ASTs & args, ContextPtr context);
static Configuration getConfiguration(ASTs & args, const ContextPtr & context);
/// Does evaluateConstantExpressionOrIdentifierAsLiteral() on all arguments.
/// If `headers(...)` argument is present, parses it and moves it to the end of the array.
/// Returns number of arguments excluding `headers(...)`.
static size_t evalArgsAndCollectHeaders(ASTs & url_function_args, HTTPHeaderEntries & header_entries, ContextPtr context);
static size_t evalArgsAndCollectHeaders(ASTs & url_function_args, HTTPHeaderEntries & header_entries, const ContextPtr & context);
static void processNamedCollectionResult(Configuration & configuration, const NamedCollection & collection);
};
@ -314,7 +330,7 @@ public:
const std::optional<FormatSettings> & format_settings_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
ContextPtr context_,
const ContextPtr & context_,
const String & compression_method_);
void read(

View File

@ -35,36 +35,43 @@ namespace ErrorCodes
}
StorageURLCluster::StorageURLCluster(
ContextPtr context_,
const ContextPtr & context,
const String & cluster_name_,
const String & uri_,
const String & format_,
const String & compression_method_,
const String & compression_method,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const StorageURL::Configuration & configuration_,
bool structure_argument_was_provided_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"), structure_argument_was_provided_)
, uri(uri_)
const StorageURL::Configuration & configuration_)
: IStorageCluster(cluster_name_, table_id_, &Poco::Logger::get("StorageURLCluster (" + table_id_.table_name + ")"))
, uri(uri_), format_name(format_)
{
context_->getRemoteHostFilter().checkURL(Poco::URI(uri));
context_->getHTTPHeaderFilter().checkHeaders(configuration_.headers);
context->getRemoteHostFilter().checkURL(Poco::URI(uri));
context->getHTTPHeaderFilter().checkHeaders(configuration_.headers);
StorageInMemoryMetadata storage_metadata;
if (columns_.empty())
{
auto columns = StorageURL::getTableStructureFromData(format_,
uri,
chooseCompressionMethod(Poco::URI(uri).getPath(), compression_method_),
configuration_.headers,
std::nullopt,
context_);
ColumnsDescription columns;
if (format_name == "auto")
std::tie(columns, format_name) = StorageURL::getTableStructureAndFormatFromData(
uri, chooseCompressionMethod(Poco::URI(uri).getPath(), compression_method), configuration_.headers, std::nullopt, context);
else
columns = StorageURL::getTableStructureFromData(
format_, uri, chooseCompressionMethod(Poco::URI(uri).getPath(), compression_method), configuration_.headers, std::nullopt, context);
storage_metadata.setColumns(columns);
}
else
{
if (format_name == "auto")
format_name = StorageURL::getTableStructureAndFormatFromData(
uri, chooseCompressionMethod(Poco::URI(uri).getPath(), compression_method), configuration_.headers, std::nullopt, context).second;
storage_metadata.setColumns(columns_);
}
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
@ -72,13 +79,14 @@ StorageURLCluster::StorageURLCluster(
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
}
void StorageURLCluster::addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context)
void StorageURLCluster::updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function urlCluster, got '{}'", queryToString(query));
TableFunctionURLCluster::addColumnsStructureToArguments(expression_list->children, structure, context);
TableFunctionURLCluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children, storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(), format_name, context);
}
RemoteQueryExecutor::Extension StorageURLCluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const

View File

@ -19,16 +19,15 @@ class StorageURLCluster : public IStorageCluster
{
public:
StorageURLCluster(
ContextPtr context_,
const ContextPtr & context,
const String & cluster_name_,
const String & uri_,
const String & format_,
const String & compression_method_,
const String & compression_method,
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const StorageURL::Configuration & configuration_,
bool structure_argument_was_provided_);
const StorageURL::Configuration & configuration_);
std::string getName() const override { return "URLCluster"; }
@ -41,11 +40,10 @@ public:
bool supportsTrivialCountOptimization() const override { return true; }
private:
void addColumnsStructureToQuery(ASTPtr & query, const String & structure, const ContextPtr & context) override;
void updateQueryToSendIfNeeded(ASTPtr & query, const StorageSnapshotPtr & storage_snapshot, const ContextPtr & context) override;
String uri;
String format_name;
String compression_method;
NamesAndTypesList virtual_columns;
};

View File

@ -59,7 +59,7 @@ std::vector<std::pair<std::string, std::string>> StorageXDBC::getReadURIParams(
const Names & /* column_names */,
const StorageSnapshotPtr & /*storage_snapshot*/,
const SelectQueryInfo & /*query_info*/,
ContextPtr /*context*/,
const ContextPtr & /*context*/,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t max_block_size) const
{
@ -70,7 +70,7 @@ std::function<void(std::ostream &)> StorageXDBC::getReadPOSTDataCallback(
const Names & column_names,
const ColumnsDescription & columns_description,
const SelectQueryInfo & query_info,
ContextPtr local_context,
const ContextPtr & local_context,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t /*max_block_size*/) const
{

View File

@ -55,7 +55,7 @@ private:
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
const SelectQueryInfo & query_info,
ContextPtr context,
const ContextPtr & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size) const override;
@ -63,7 +63,7 @@ private:
const Names & column_names,
const ColumnsDescription & columns_description,
const SelectQueryInfo & query_info,
ContextPtr context,
const ContextPtr & context,
QueryProcessingStage::Enum & processed_stage,
size_t max_block_size) const override;

View File

@ -4,7 +4,6 @@
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Storages/StorageS3Cluster.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <TableFunctions/ITableFunction.h>
#include <TableFunctions/TableFunctionAzureBlobStorage.h>
@ -29,14 +28,14 @@ public:
String getName() const override = 0;
String getSignature() const override = 0;
static void addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context)
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context)
{
if (args.empty())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Unexpected empty list of arguments for {}Cluster table function", Base::name);
ASTPtr cluster_name_arg = args.front();
args.erase(args.begin());
Base::addColumnsStructureToArguments(args, desired_structure, context);
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
args.insert(args.begin(), cluster_name_arg);
}

View File

@ -27,14 +27,14 @@ void ITableFunctionFileLike::parseFirstArguments(const ASTPtr & arg, const Conte
filename = checkAndGetLiteralArgument<String>(arg, "source");
}
String ITableFunctionFileLike::getFormatFromFirstArgument()
std::optional<String> ITableFunctionFileLike::tryGetFormatFromFirstArgument()
{
return FormatFactory::instance().getFormatFromFileName(filename, true);
return FormatFactory::instance().tryGetFormatFromFileName(filename);
}
bool ITableFunctionFileLike::supportsReadingSubsetOfColumns(const ContextPtr & context)
{
return FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format, context);
return format != "auto" && FormatFactory::instance().checkIfFormatSupportsSubsetOfColumns(format, context);
}
void ITableFunctionFileLike::parseArguments(const ASTPtr & ast_function, ContextPtr context)
@ -63,7 +63,10 @@ void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr &
format = checkAndGetLiteralArgument<String>(args[1], "format");
if (format == "auto")
format = getFormatFromFirstArgument();
{
if (auto format_from_first_argument = tryGetFormatFromFirstArgument())
format = *format_from_first_argument;
}
if (args.size() > 2)
{
@ -79,34 +82,37 @@ void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr &
compression_method = checkAndGetLiteralArgument<String>(args[3], "compression_method");
}
void ITableFunctionFileLike::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr &)
void ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context)
{
if (args.empty() || args.size() > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), args.size());
auto format_literal = std::make_shared<ASTLiteral>(format);
auto structure_literal = std::make_shared<ASTLiteral>(structure);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// f(filename)
if (args.size() == 1)
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
args.push_back(structure_literal);
}
/// f(filename, format)
else if (args.size() == 2)
{
if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args.back() = format_literal;
args.push_back(structure_literal);
}
/// f(filename, format, 'auto')
else if (args.size() == 3)
/// f(filename, format, structure) or f(filename, format, structure, compression)
else if (args.size() >= 3)
{
args.back() = structure_literal;
}
/// f(filename, format, 'auto', compression)
else if (args.size() == 4)
{
args[args.size() - 2] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
}

View File

@ -31,7 +31,7 @@ public:
static size_t getMaxNumberOfArguments() { return 4; }
static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr &);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &);
protected:
@ -39,7 +39,7 @@ protected:
virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context);
virtual void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context);
virtual String getFormatFromFirstArgument();
virtual std::optional<String> tryGetFormatFromFirstArgument();
String filename;
String path_to_archive;

View File

@ -58,7 +58,7 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const
configuration.blobs_paths = {configuration.blob_path};
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true);
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path);
}
else
{
@ -155,7 +155,7 @@ void TableFunctionAzureBlobStorage::parseArgumentsImpl(ASTs & engine_args, const
configuration.blobs_paths = {configuration.blob_path};
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(configuration.blob_path, true);
configuration.format = FormatFactory::instance().tryGetFormatFromFileName(configuration.blob_path).value_or("auto");
}
}
@ -174,15 +174,24 @@ void TableFunctionAzureBlobStorage::parseArguments(const ASTPtr & ast_function,
parseArgumentsImpl(args, context);
}
void TableFunctionAzureBlobStorage::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context)
void TableFunctionAzureBlobStorage::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
@ -191,65 +200,126 @@ void TableFunctionAzureBlobStorage::addColumnsStructureToArguments(ASTs & args,
"Storage Azure requires 3 to 7 arguments: "
"AzureBlobStorage(connection_string|storage_account_url, container_name, blobpath, [account_name, account_key, format, compression, structure])");
auto format_literal = std::make_shared<ASTLiteral>(format);
auto structure_literal = std::make_shared<ASTLiteral>(structure);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
auto is_format_arg
= [](const std::string & s) -> bool { return s == "auto" || FormatFactory::instance().getAllFormats().contains(s); };
/// (connection_string, container_name, blobpath)
if (args.size() == 3)
{
/// Add format=auto & compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
/// Add compression = "auto" before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
/// (connection_string, container_name, blobpath, structure) or
/// (connection_string, container_name, blobpath, format)
/// We can distinguish them by looking at the 4-th argument: check if it's format name or not.
else if (args.size() == 4)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/account_name/structure");
/// (..., format) -> (..., format, compression, structure)
if (is_format_arg(fourth_arg))
{
if (fourth_arg == "auto")
args[3] = format_literal;
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
/// (..., structure) -> (..., format, compression, structure)
else
{
args.back() = structure_literal;
auto structure_arg = args.back();
args[3] = format_literal;
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
if (fourth_arg == "auto")
args.push_back(structure_literal);
else
args.push_back(structure_arg);
}
}
/// (connection_string, container_name, blobpath, format, compression) or
/// (storage_account_url, container_name, blobpath, account_name, account_key)
/// We can distinguish them by looking at the 4-th argument: check if it's format name or not.
else if (args.size() == 5)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/account_name");
if (!is_format_arg(fourth_arg))
/// (..., format, compression) -> (..., format, compression, structure)
if (is_format_arg(fourth_arg))
{
/// Add format=auto & compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(std::make_shared<ASTLiteral>("auto"));
if (fourth_arg == "auto")
args[3] = format_literal;
args.push_back(structure_literal);
}
args.push_back(structure_literal);
}
else if (args.size() == 6)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/account_name");
if (!is_format_arg(fourth_arg))
/// (..., account_name, account_key) -> (..., account_name, account_key, format, compression, structure)
else
{
args.push_back(format_literal);
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
}
/// (connection_string, container_name, blobpath, format, compression, structure) or
/// (storage_account_url, container_name, blobpath, account_name, account_key, structure) or
/// (storage_account_url, container_name, blobpath, account_name, account_key, format)
else if (args.size() == 6)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/account_name");
auto sixth_arg = checkAndGetLiteralArgument<String>(args[5], "format/structure");
/// (..., format, compression, structure)
if (is_format_arg(fourth_arg))
{
if (fourth_arg == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "structure") == "auto")
args[5] = structure_literal;
}
/// (..., account_name, account_key, format) -> (..., account_name, account_key, format, compression, structure)
else if (is_format_arg(sixth_arg))
{
if (sixth_arg == "auto")
args[5] = format_literal;
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
/// (..., account_name, account_key, structure) -> (..., account_name, account_key, format, compression, structure)
else
{
args.back() = structure_literal;
auto structure_arg = args.back();
args[5] = format_literal;
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
if (sixth_arg == "auto")
args.push_back(structure_literal);
else
args.push_back(structure_arg);
}
}
/// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression)
else if (args.size() == 7)
{
/// (..., format, compression) -> (..., format, compression, structure)
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = format_literal;
args.push_back(structure_literal);
}
/// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure)
else if (args.size() == 8)
{
args.back() = structure_literal;
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = format_literal;
if (checkAndGetLiteralArgument<String>(args[7], "structure") == "auto")
args[7] = structure_literal;
}
}
}
@ -263,7 +333,9 @@ ColumnsDescription TableFunctionAzureBlobStorage::getActualTableStructure(Contex
auto settings = StorageAzureBlob::createSettings(context);
auto object_storage = std::make_unique<AzureObjectStorage>("AzureBlobStorageTableFunction", std::move(client), std::move(settings));
return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context, false);
if (configuration.format == "auto")
return StorageAzureBlob::getTableStructureAndFormatFromData(object_storage.get(), configuration, std::nullopt, context).first;
return StorageAzureBlob::getTableStructureFromData(object_storage.get(), configuration, std::nullopt, context);
}
return parseColumnsListFromString(configuration.structure, context);

View File

@ -55,7 +55,7 @@ public:
virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context);
static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context);
protected:

View File

@ -21,9 +21,8 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl(
{
StoragePtr storage;
ColumnsDescription columns;
bool structure_argument_was_provided = configuration.structure != "auto";
if (structure_argument_was_provided)
if (configuration.structure != "auto")
{
columns = parseColumnsListFromString(configuration.structure, context);
}
@ -59,8 +58,7 @@ StoragePtr TableFunctionAzureBlobStorageCluster::executeImpl(
StorageID(getDatabaseName(), table_name),
columns,
ConstraintsDescription{},
context,
structure_argument_was_provided);
context);
}
storage->startup();

View File

@ -54,12 +54,12 @@ void TableFunctionFile::parseFirstArguments(const ASTPtr & arg, const ContextPtr
throw Exception(ErrorCodes::BAD_ARGUMENTS, "The first argument of table function '{}' mush be path or file descriptor", getName());
}
String TableFunctionFile::getFormatFromFirstArgument()
std::optional<String> TableFunctionFile::tryGetFormatFromFirstArgument()
{
if (fd >= 0)
return FormatFactory::instance().getFormatFromFileDescriptor(fd);
return FormatFactory::instance().tryGetFormatFromFileDescriptor(fd);
else
return FormatFactory::instance().getFormatFromFileName(filename, true);
return FormatFactory::instance().tryGetFormatFromFileName(filename);
}
StoragePtr TableFunctionFile::getStorage(const String & source,
@ -104,10 +104,11 @@ ColumnsDescription TableFunctionFile::getActualTableStructure(ContextPtr context
archive_info
= StorageFile::getArchiveInfo(path_to_archive, filename, context->getUserFilesPath(), context, total_bytes_to_read);
if (format == "auto")
return StorageFile::getTableStructureAndFormatFromFile(paths, compression_method, std::nullopt, context, archive_info).first;
return StorageFile::getTableStructureFromFile(format, paths, compression_method, std::nullopt, context, archive_info);
}
return parseColumnsListFromString(structure, context);
}

View File

@ -27,7 +27,7 @@ public:
protected:
int fd = -1;
void parseFirstArguments(const ASTPtr & arg, const ContextPtr & context) override;
String getFormatFromFirstArgument() override;
std::optional<String> tryGetFormatFromFirstArgument() override;
private:
StoragePtr getStorage(

View File

@ -43,8 +43,7 @@ StoragePtr TableFunctionFileCluster::getStorage(
compression_method,
StorageID(getDatabaseName(), table_name),
columns,
ConstraintsDescription{},
structure != "auto");
ConstraintsDescription{});
}
return storage;

View File

@ -33,7 +33,9 @@ namespace ErrorCodes
namespace
{
/* format(format_name, data) - ...
/* format(format_name, structure, data) - parses data according to the specified format and structure.
* format(format_name, data) - infers the schema from the data and parses it according to the specified format.
* format(data) - detects the format, infers the schema and parses data according to inferred format and structure.
*/
class TableFunctionFormat : public ITableFunction
{
@ -49,11 +51,11 @@ private:
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
void parseArguments(const ASTPtr & ast_function, ContextPtr context) override;
Block parseData(ColumnsDescription columns, ContextPtr context) const;
Block parseData(const ColumnsDescription & columns, const String & format_name, const ContextPtr & context) const;
String format;
String data;
String format = "auto";
String structure = "auto";
String data;
};
void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr context)
@ -65,14 +67,15 @@ void TableFunctionFormat::parseArguments(const ASTPtr & ast_function, ContextPtr
ASTs & args = args_func.at(0)->children;
if (args.size() != 2 && args.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires 2 or 3 arguments: format, [structure], data", getName());
if (args.empty() || args.size() > 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, "Table function '{}' requires from 1 to 3 arguments: [format, [structure]], data", getName());
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
format = checkAndGetLiteralArgument<String>(args[0], "format");
data = checkAndGetLiteralArgument<String>(args.back(), "data");
if (args.size() > 1)
format = checkAndGetLiteralArgument<String>(args[0], "format");
if (args.size() == 3)
structure = checkAndGetLiteralArgument<String>(args[1], "structure");
}
@ -82,19 +85,21 @@ ColumnsDescription TableFunctionFormat::getActualTableStructure(ContextPtr conte
if (structure == "auto")
{
SingleReadBufferIterator read_buffer_iterator(std::make_unique<ReadBufferFromString>(data));
return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, false, context);
if (format == "auto")
return detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context).first;
return readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context);
}
return parseColumnsListFromString(structure, context);
}
Block TableFunctionFormat::parseData(ColumnsDescription columns, ContextPtr context) const
Block TableFunctionFormat::parseData(const ColumnsDescription & columns, const String & format_name, const ContextPtr & context) const
{
Block block;
for (const auto & name_and_type : columns.getAllPhysical())
block.insert({name_and_type.type->createColumn(), name_and_type.type, name_and_type.name});
auto read_buf = std::make_unique<ReadBufferFromString>(data);
auto input_format = context->getInputFormat(format, *read_buf, block, context->getSettingsRef().max_block_size);
auto input_format = context->getInputFormat(format_name, *read_buf, block, context->getSettingsRef().max_block_size);
QueryPipelineBuilder builder;
builder.init(Pipe(input_format));
if (columns.hasDefaults())
@ -120,10 +125,24 @@ Block TableFunctionFormat::parseData(ColumnsDescription columns, ContextPtr cont
return concatenateBlocks(blocks);
}
StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool is_insert_query) const
StoragePtr TableFunctionFormat::executeImpl(const ASTPtr & /*ast_function*/, ContextPtr context, const std::string & table_name, ColumnsDescription /*cached_columns*/, bool /*is_insert_query*/) const
{
auto columns = getActualTableStructure(context, is_insert_query);
Block res_block = parseData(columns, context);
ColumnsDescription columns;
String format_name = format;
if (structure == "auto")
{
SingleReadBufferIterator read_buffer_iterator(std::make_unique<ReadBufferFromString>(data));
if (format_name == "auto")
std::tie(columns, format_name) = detectFormatAndReadSchema(std::nullopt, read_buffer_iterator, context);
else
columns = readSchemaFromFormat(format, std::nullopt, read_buffer_iterator, context);
}
else
{
columns = parseColumnsListFromString(structure, context);
}
Block res_block = parseData(columns, format_name, context);
auto res = std::make_shared<StorageValues>(StorageID(getDatabaseName(), table_name), columns, res_block);
res->startup();
return res;

View File

@ -33,6 +33,8 @@ ColumnsDescription TableFunctionHDFS::getActualTableStructure(ContextPtr context
if (structure == "auto")
{
context->checkAccess(getSourceAccessType());
if (format == "auto")
return StorageHDFS::getTableStructureAndFormatFromData(filename, compression_method, context).first;
return StorageHDFS::getTableStructureFromData(format, filename, compression_method, context);
}

View File

@ -45,8 +45,7 @@ StoragePtr TableFunctionHDFSCluster::getStorage(
format,
columns,
ConstraintsDescription{},
compression_method,
structure != "auto");
compression_method);
}
return storage;
}

View File

@ -61,12 +61,11 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
if (configuration.format == "auto")
{
String file_path = named_collection->getOrDefault<String>("filename", Poco::URI(named_collection->get<String>("url")).getPath());
configuration.format = FormatFactory::instance().getFormatFromFileName(file_path, true);
configuration.format = FormatFactory::instance().tryGetFormatFromFileName(file_path).value_or("auto");
}
}
else
{
size_t count = StorageURL::evalArgsAndCollectHeaders(args, configuration.headers_from_ast, context);
if (count == 0 || count > 7)
@ -216,7 +215,7 @@ void TableFunctionS3::parseArgumentsImpl(ASTs & args, const ContextPtr & context
configuration.auth_settings.no_sign_request = no_sign_request;
if (configuration.format == "auto")
configuration.format = FormatFactory::instance().getFormatFromFileName(Poco::URI(url).getPath(), true);
configuration.format = FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(url).getPath()).value_or("auto");
}
configuration.keys = {configuration.url.key};
@ -238,15 +237,24 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
parseArgumentsImpl(args, context);
}
void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context)
void TableFunctionS3::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
@ -256,23 +264,25 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
if (count == 0 || count > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), count);
auto format_literal = std::make_shared<ASTLiteral>(format);
auto structure_literal = std::make_shared<ASTLiteral>(structure);
/// s3(s3_url)
/// s3(s3_url) -> s3(s3_url, format, structure)
if (count == 1)
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
args.push_back(structure_literal);
}
/// s3(s3_url, format) or s3(s3_url, NOSIGN)
/// s3(s3_url, format) -> s3(s3_url, format, structure) or
/// s3(s3_url, NOSIGN) -> s3(s3_url, NOSIGN, format, structure)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN or not.
else if (count == 2)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// If there is NOSIGN, add format=auto before structure.
if (boost::iequals(second_arg, "NOSIGN"))
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
else if (second_arg == "auto")
args.back() = format_literal;
args.push_back(structure_literal);
}
/// s3(source, format, structure) or
@ -282,18 +292,25 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
else if (count == 3)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// s3(source, NOSIGN, format) -> s3(source, NOSIGN, format, structure)
if (boost::iequals(second_arg, "NOSIGN"))
{
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args.back() = format_literal;
args.push_back(structure_literal);
}
/// s3(source, format, structure)
else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))
{
args[count - 1] = structure_literal;
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
/// s3(source, access_key_id, access_key_id) -> s3(source, access_key_id, access_key_id, format, structure)
else
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(format_literal);
args.push_back(structure_literal);
}
}
@ -304,16 +321,27 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
else if (count == 4)
{
auto second_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// s3(source, NOSIGN, format, structure)
if (boost::iequals(second_arg, "NOSIGN"))
{
args[count - 1] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[3], "structure") == "auto")
args[3] = structure_literal;
}
/// s3(source, format, structure, compression_method)
else if (second_arg == "auto" || FormatFactory::instance().getAllFormats().contains(second_arg))
{
args[count - 2] = structure_literal;
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
/// s3(source, access_key_id, access_key_id, format) -> s3(source, access_key_id, access_key_id, format, structure)
else
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
args.push_back(structure_literal);
}
}
@ -323,19 +351,30 @@ void TableFunctionS3::addColumnsStructureToArguments(ASTs & args, const String &
else if (count == 5)
{
auto sedond_arg = checkAndGetLiteralArgument<String>(args[1], "format/NOSIGN");
/// s3(source, NOSIGN, format, structure, compression_method)
if (boost::iequals(sedond_arg, "NOSIGN"))
{
args[count - 2] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[3], "structure") == "auto")
args[3] = structure_literal;
}
/// s3(source, access_key_id, access_key_id, format, structure)
else
{
args[count - 1] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
}
/// s3(source, access_key_id, secret_access_key, format, structure, compression)
else if (count == 6)
{
args[count - 2] = structure_literal;
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
}
}
@ -346,6 +385,9 @@ ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context,
{
context->checkAccess(getSourceAccessType());
configuration.update(context);
if (configuration.format == "auto")
return StorageS3::getTableStructureAndFormatFromData(configuration, std::nullopt, context).first;
return StorageS3::getTableStructureFromData(configuration, std::nullopt, context);
}

View File

@ -57,7 +57,7 @@ public:
virtual void parseArgumentsImpl(ASTs & args, const ContextPtr & context);
static void addColumnsStructureToArguments(ASTs & args, const String & structure, const ContextPtr & context);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context);
protected:

View File

@ -21,9 +21,8 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
{
StoragePtr storage;
ColumnsDescription columns;
bool structure_argument_was_provided = configuration.structure != "auto";
if (structure_argument_was_provided)
if (configuration.structure != "auto")
{
columns = parseColumnsListFromString(configuration.structure, context);
}
@ -53,8 +52,7 @@ StoragePtr TableFunctionS3Cluster::executeImpl(
StorageID(getDatabaseName(), table_name),
columns,
ConstraintsDescription{},
context,
structure_argument_was_provided);
context);
}
storage->startup();

View File

@ -55,7 +55,7 @@ void TableFunctionURL::parseArgumentsImpl(ASTs & args, const ContextPtr & contex
format = configuration.format;
if (format == "auto")
format = FormatFactory::instance().getFormatFromFileName(Poco::URI(filename).getPath(), true);
format = FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(filename).getPath()).value_or("auto");
StorageURL::evalArgsAndCollectHeaders(args, configuration.headers, context);
}
@ -78,15 +78,24 @@ void TableFunctionURL::parseArgumentsImpl(ASTs & args, const ContextPtr & contex
}
}
void TableFunctionURL::addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context)
void TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context)
{
if (tryGetNamedCollectionWithOverrides(args, context))
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
/// In case of named collection, just add key-value pair "structure='...'"
/// at the end of arguments to override existed structure.
ASTs equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(desired_structure)};
auto equal_func = makeASTFunction("equals", std::move(equal_func_args));
args.push_back(equal_func);
/// In case of named collection, just add key-value pairs "format='...', structure='...'"
/// at the end of arguments to override existed format and structure with "auto" values.
if (collection->getOrDefault<String>("format", "auto") == "auto")
{
ASTs format_equal_func_args = {std::make_shared<ASTIdentifier>("format"), std::make_shared<ASTLiteral>(format_)};
auto format_equal_func = makeASTFunction("equals", std::move(format_equal_func_args));
args.push_back(format_equal_func);
}
if (collection->getOrDefault<String>("structure", "auto") == "auto")
{
ASTs structure_equal_func_args = {std::make_shared<ASTIdentifier>("structure"), std::make_shared<ASTLiteral>(structure_)};
auto structure_equal_func = makeASTFunction("equals", std::move(structure_equal_func_args));
args.push_back(structure_equal_func);
}
}
else
{
@ -101,7 +110,7 @@ void TableFunctionURL::addColumnsStructureToArguments(ASTs & args, const String
args.pop_back();
}
ITableFunctionFileLike::addColumnsStructureToArguments(args, desired_structure, context);
ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
if (headers_ast)
args.push_back(headers_ast);
@ -131,6 +140,14 @@ ColumnsDescription TableFunctionURL::getActualTableStructure(ContextPtr context,
if (structure == "auto")
{
context->checkAccess(getSourceAccessType());
if (format == "auto")
return StorageURL::getTableStructureAndFormatFromData(
filename,
chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method),
configuration.headers,
std::nullopt,
context).first;
return StorageURL::getTableStructureFromData(format,
filename,
chooseCompressionMethod(Poco::URI(filename).getPath(), compression_method),
@ -148,9 +165,9 @@ std::unordered_set<String> TableFunctionURL::getVirtualsToCheckBeforeUsingStruct
return {virtual_column_names.begin(), virtual_column_names.end()};
}
String TableFunctionURL::getFormatFromFirstArgument()
std::optional<String> TableFunctionURL::tryGetFormatFromFirstArgument()
{
return FormatFactory::instance().getFormatFromFileName(Poco::URI(filename).getPath(), true);
return FormatFactory::instance().tryGetFormatFromFileName(Poco::URI(filename).getPath());
}
void registerTableFunctionURL(TableFunctionFactory & factory)

View File

@ -34,7 +34,7 @@ public:
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
static void addColumnsStructureToArguments(ASTs & args, const String & desired_structure, const ContextPtr & context);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context);
std::unordered_set<String> getVirtualsToCheckBeforeUsingStructureHint() const override;
@ -53,8 +53,7 @@ private:
const char * getStorageTypeName() const override { return "URL"; }
String getFormatFromFirstArgument() override;
std::optional<String> tryGetFormatFromFirstArgument() override;
};
}

View File

@ -40,8 +40,7 @@ StoragePtr TableFunctionURLCluster::getStorage(
StorageID(getDatabaseName(), table_name),
getActualTableStructure(context, /* is_insert_query */ true),
ConstraintsDescription{},
configuration,
structure != "auto");
configuration);
}
return storage;
}

View File

@ -123,3 +123,91 @@ def test_no_such_files(started_cluster):
distributed = node.query(get_query("*", True, "3,4"))
assert TSV(local) == TSV(distributed)
def test_schema_inference(started_cluster):
node = started_cluster.instances["s0_0_0"]
expected_result = node.query(
"select * from file('file*.csv', 'CSV', 's String, i UInt32') ORDER BY (i, s)"
)
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv') ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', CSV) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', auto, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', CSV, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', auto, auto, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file*.csv', CSV, auto, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
def test_format_detection(started_cluster):
for node_name in ("s0_0_0", "s0_0_1", "s0_1_0"):
for i in range(1, 3):
started_cluster.instances[node_name].query(
f"""
INSERT INTO TABLE FUNCTION file(
'file_for_format_detection_{i}', 'CSV', 's String, i UInt32') VALUES ('file{i}',{i})
"""
)
node = started_cluster.instances["s0_0_0"]
expected_result = node.query(
"select * from file('file_for_format_detection*', 'CSV', 's String, i UInt32') ORDER BY (i, s)"
)
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*') ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*', auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*', auto, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*', auto, 's String, i UInt32') ORDER BY (i, s)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*', auto, auto, auto) ORDER BY (c1, c2)"
)
assert result == expected_result
result = node.query(
"select * from fileCluster('my_cluster', 'file_for_format_detection*', auto, 's String, i UInt32', auto) ORDER BY (i, s)"
)
assert result == expected_result

View File

@ -35,7 +35,9 @@ def create_buckets_s3(cluster):
# Make all files a bit different
for number in range(100 + file_number):
data.append([str(number + file_number) * 10, number + file_number])
data.append(
["str_" + str(number + file_number) * 10, number + file_number]
)
writer = csv.writer(f)
writer.writerows(data)
@ -427,3 +429,33 @@ def test_cluster_with_named_collection(started_cluster):
)
assert TSV(pure_s3) == TSV(s3_cluster)
def test_cluster_format_detection(started_cluster):
node = started_cluster.instances["s0_0_0"]
expected_desc_result = node.query(
"desc s3('http://minio1:9001/root/data/generated/*', 'minio', 'minio123', 'CSV')"
)
desc_result = node.query(
"desc s3('http://minio1:9001/root/data/generated/*', 'minio', 'minio123')"
)
assert expected_desc_result == desc_result
expected_result = node.query(
"SELECT * FROM s3('http://minio1:9001/root/data/generated/*', 'minio', 'minio123', 'CSV', 'a String, b UInt64') order by a, b"
)
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/generated/*', 'minio', 'minio123') order by c1, c2"
)
assert result == expected_result
result = node.query(
"SELECT * FROM s3Cluster(cluster_simple, 'http://minio1:9001/root/data/generated/*', 'minio', 'minio123', auto, 'a String, b UInt64') order by a, b"
)
assert result == expected_result

View File

@ -1250,3 +1250,73 @@ def test_size_virtual_column(cluster):
result
== "test_size_virtual_column1.tsv\t2\ntest_size_virtual_column2.tsv\t3\ntest_size_virtual_column3.tsv\t4\n"
)
def test_format_detection(cluster):
node = cluster.instances["node"]
storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]
account_name = "devstoreaccount1"
account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(0)",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String') select number as x, 'str_' || toString(number) from numbers(10)",
)
expected_desc_result = azure_query(
node,
f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'auto')",
)
desc_result = azure_query(
node,
f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}')",
)
assert expected_desc_result == desc_result
expected_result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt64, y String')",
)
result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}')",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', auto, auto, 'x UInt64, y String')",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection{{0,1}}', '{account_name}', '{account_key}')",
)
assert result == expected_result
node.query(f"system drop schema cache for hdfs")
result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection{{0,1}}', '{account_name}', '{account_key}')",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection{{0,1}}', '{account_name}', '{account_key}')",
)
assert result == expected_result

View File

@ -262,3 +262,72 @@ def test_partition_parallel_reading_with_cluster(cluster):
)
assert azure_cluster == "3\n"
def test_format_detection(cluster):
node = cluster.instances["node_0"]
storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]
account_name = "devstoreaccount1"
account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection0', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10)",
)
azure_query(
node,
f"INSERT INTO TABLE FUNCTION azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') select number as x, 'str_' || toString(number) from numbers(10, 10)",
)
expected_desc_result = azure_query(
node,
f"desc azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'auto')",
)
desc_result = azure_query(
node,
f"desc azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}')",
)
assert expected_desc_result == desc_result
expected_result = azure_query(
node,
f"select * from azureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', 'JSONEachRow', 'auto', 'x UInt32, y String') order by x",
)
result = azure_query(
node,
f"select * from azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}') order by x",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', auto) order by x",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', auto, auto) order by x",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', 'x UInt32, y String') order by x",
)
assert result == expected_result
result = azure_query(
node,
f"select * from azureBlobStorageCluster('simple_cluster', '{storage_account_url}', 'cont', 'test_format_detection*', '{account_name}', '{account_key}', auto, auto, 'x UInt32, y String') order by x",
)
assert result == expected_result

View File

@ -1047,6 +1047,74 @@ def test_union_schema_inference_mode(started_cluster):
assert "Cannot extract table structure" in error
def test_format_detection(started_cluster):
node = started_cluster.instances["node1"]
node.query(
"insert into function hdfs('hdfs://hdfs1:9000/test_format_detection0', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(0)"
)
node.query(
"insert into function hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow) select number as x, 'str_' || toString(number) as y from numbers(10)"
)
expected_desc_result = node.query(
"desc hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow)"
)
desc_result = node.query("desc hdfs('hdfs://hdfs1:9000/test_format_detection1')")
assert expected_desc_result == desc_result
expected_result = node.query(
"select * from hdfs('hdfs://hdfs1:9000/test_format_detection1', JSONEachRow, 'x UInt64, y String') order by x, y"
)
result = node.query(
"select * from hdfs('hdfs://hdfs1:9000/test_format_detection1') order by x, y"
)
assert expected_result == result
result = node.query(
"select * from hdfs('hdfs://hdfs1:9000/test_format_detection1', auto, 'x UInt64, y String') order by x, y"
)
assert expected_result == result
result = node.query(
"select * from hdfs('hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y"
)
assert expected_result == result
node.query("system drop schema cache for hdfs")
result = node.query(
"select * from hdfs('hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y"
)
assert expected_result == result
result = node.query(
"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}') order by x, y"
)
assert expected_result == result
result = node.query(
"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}', auto, auto) order by x, y"
)
assert expected_result == result
result = node.query(
"select * from hdfsCluster(test_cluster_two_shards, 'hdfs://hdfs1:9000/test_format_detection{0,1}', auto, 'x UInt64, y String') order by x, y"
)
assert expected_result == result
if __name__ == "__main__":
cluster.start()
input("Cluster created, press any key to destroy...")

View File

@ -2194,3 +2194,57 @@ def test_union_schema_inference_mode(started_cluster):
f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_union_schema_inference{{1,2,3,4}}.jsonl') settings schema_inference_mode='union', describe_compact_output=1 format TSV"
)
assert "Cannot extract table structure" in error
def test_s3_format_detection(started_cluster):
bucket = started_cluster.minio_bucket
instance = started_cluster.instances["dummy"]
instance.query(
f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection0', 'JSONEachRow', 'x UInt64, y String') select number, 'str_' || toString(number) from numbers(0) settings s3_truncate_on_insert=1"
)
instance.query(
f"insert into table function s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1', 'JSONEachRow', 'x UInt64, y String') select number, 'str_' || toString(number) from numbers(5) settings s3_truncate_on_insert=1"
)
expected_result = instance.query(
f"select * from s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1', 'JSONEachRow', 'x UInt64, y String')"
)
expected_desc_result = instance.query(
f"desc s3('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1', 'JSONEachRow')"
)
for engine in ["s3", "url"]:
desc_result = instance.query(
f"desc {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1')"
)
assert desc_result == expected_desc_result
result = instance.query(
f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1')"
)
assert result == expected_result
result = instance.query(
f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection1', auto, 'x UInt64, y String')"
)
assert result == expected_result
result = instance.query(
f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection{{0,1}}', auto, 'x UInt64, y String')"
)
assert result == expected_result
instance.query(f"system drop schema cache for {engine}")
result = instance.query(
f"select * from {engine}('http://{started_cluster.minio_host}:{started_cluster.minio_port}/{bucket}/test_format_detection{{0,1}}', auto, 'x UInt64, y String')"
)
assert result == expected_result

View File

@ -0,0 +1,123 @@
Parquet
a Nullable(UInt64)
b Nullable(String)
c Array(Nullable(UInt64))
d Tuple(\n a Nullable(UInt64),\n b Nullable(String))
ORC
a Nullable(Int64)
b Nullable(String)
c Array(Nullable(Int64))
d Tuple(\n a Nullable(Int64),\n b Nullable(String))
Arrow
a Nullable(UInt64)
b Nullable(String)
c Array(Nullable(UInt64))
d Tuple(\n a Nullable(UInt64),\n b Nullable(String))
ArrowStream
a Nullable(UInt64)
b Nullable(String)
c Array(Nullable(UInt64))
d Tuple(\n a Nullable(UInt64),\n b Nullable(String))
Avro
a Int64
b String
c Array(Int64)
d Tuple(\n a Int64,\n b String)
Native
a UInt64
b String
c Array(UInt64)
d Tuple(\n a UInt64,\n b String)
BSONEachRow
a Nullable(Int64)
b Nullable(String)
c Array(Nullable(Int64))
d Tuple(\n a Nullable(Int64),\n b Nullable(String))
JSONCompact
a UInt64
b String
c Array(UInt64)
d Tuple(\n a UInt64,\n b String)
Values
c1 Nullable(UInt64)
c2 Nullable(String)
c3 Array(Nullable(UInt64))
c4 Tuple(Nullable(UInt64), Nullable(String))
TSKV
a Nullable(String)
b Nullable(String)
c Array(Nullable(UInt64))
d Nullable(String)
JSONObjectEachRow
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
JSONColumns
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
JSONCompactColumns
c1 Nullable(String)
c2 Nullable(String)
c3 Array(Nullable(String))
c4 Tuple(\n a Nullable(String),\n b Nullable(String))
JSONCompact
a UInt64
b String
c Array(UInt64)
d Tuple(\n a UInt64,\n b String)
JSON
a UInt64
b String
c Array(UInt64)
d Tuple(\n a UInt64,\n b String)
TSV
c1 Nullable(UInt64)
c2 Nullable(String)
c3 Array(Nullable(UInt64))
c4 Tuple(Nullable(UInt64), Nullable(String))
CSV
c1 Nullable(UInt64)
c2 Nullable(String)
c3 Array(Nullable(UInt64))
c4 Nullable(UInt64)
c5 Nullable(String)
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a UInt64
b String
c Array(UInt64)
d Tuple(\n a UInt64,\n b String)
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
1
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))
a Nullable(String)
b Nullable(String)
c Array(Nullable(String))
d Tuple(\n a Nullable(String),\n b Nullable(String))

View File

@ -0,0 +1,46 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
DATA_FILE=$CLICKHOUSE_TEST_UNIQUE_NAME.data
for format in Parquet ORC Arrow ArrowStream Avro Native BSONEachRow JSONCompact Values TSKV JSONObjectEachRow JSONColumns JSONCompactColumns JSONCompact JSON TSV CSV
do
echo $format
$CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, b String, c Array(UInt64), d Tuple(a UInt64, b String)', 42) limit 10 format $format" > $DATA_FILE
$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE')"
done
rm $DATA_FILE
$CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, b String, c Array(UInt64), d Tuple(a UInt64, b String)', 42) limit 10 format JSONEachRow" > $DATA_FILE.jsonl
$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE*')"
$CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, b String, c Array(UInt64), d Tuple(a UInt64, b String)', 42) limit 10 format JSONEachRow" > $DATA_FILE
$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE', auto, 'a UInt64, b String, c Array(UInt64), d Tuple(a UInt64, b String)')"
$CLICKHOUSE_LOCAL -nmq "
desc file('$DATA_FILE');
desc file('$DATA_FILE');
"
$CLICKHOUSE_LOCAL -nmq "
desc file('$DATA_FILE', JSONEachRow);
desc file('$DATA_FILE');
"
touch $DATA_FILE.1
$CLICKHOUSE_LOCAL -q "select * from generateRandom('a UInt64, b String, c Array(UInt64), d Tuple(a UInt64, b String)', 42) limit 10 format JSONEachRow" > $DATA_FILE.2
$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE.{1,2}')"
$CLICKHOUSE_LOCAL -q "desc file('$DATA_FILE.{1,2}') settings schema_inference_mode='union'" 2>&1 | grep -c "CANNOT_DETECT_FORMAT"
$CLICKHOUSE_LOCAL -nmq "
desc file('$DATA_FILE.2');
desc file('$DATA_FILE.{1,2}');
"
rm $DATA_FILE*