mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-23 16:12:01 +00:00
Merge pull request #47496 from iiiuwioajdks/allow_empty
Allow empty column names in CSVWithNames/TSVWithNames
This commit is contained in:
commit
3460667cac
@ -1,13 +1,13 @@
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeArray.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeLowCardinality.h>
|
||||
#include <DataTypes/DataTypeMap.h>
|
||||
#include <DataTypes/DataTypeNullable.h>
|
||||
#include <DataTypes/DataTypeTuple.h>
|
||||
#include <Formats/ReadSchemaUtils.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Common/assert_cast.h>
|
||||
#include <Interpreters/Context.h>
|
||||
#include <Processors/Formats/ISchemaReader.h>
|
||||
#include <Storages/IStorage.h>
|
||||
#include <Common/assert_cast.h>
|
||||
|
||||
namespace DB
|
||||
{
|
||||
@ -20,8 +20,7 @@ namespace ErrorCodes
|
||||
extern const int CANNOT_EXTRACT_TABLE_STRUCTURE;
|
||||
}
|
||||
|
||||
static std::optional<NamesAndTypesList> getOrderedColumnsList(
|
||||
const NamesAndTypesList & columns_list, const Names & columns_order_hint)
|
||||
static std::optional<NamesAndTypesList> getOrderedColumnsList(const NamesAndTypesList & columns_list, const Names & columns_order_hint)
|
||||
{
|
||||
if (columns_list.size() != columns_order_hint.size())
|
||||
return {};
|
||||
@ -65,7 +64,8 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
e.addMessage(
|
||||
fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
}
|
||||
@ -73,7 +73,8 @@ ColumnsDescription readSchemaFromFormat(
|
||||
{
|
||||
std::string exception_messages;
|
||||
SchemaReaderPtr schema_reader;
|
||||
size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference : context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference;
|
||||
size_t max_rows_to_read = format_settings ? format_settings->max_rows_to_read_for_schema_inference
|
||||
: context->getSettingsRef().input_format_max_rows_to_read_for_schema_inference;
|
||||
size_t iterations = 0;
|
||||
ColumnsDescription cached_columns;
|
||||
while (true)
|
||||
@ -88,8 +89,8 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format(
|
||||
"Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
e.addMessage(
|
||||
fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
@ -109,7 +110,8 @@ ColumnsDescription readSchemaFromFormat(
|
||||
auto exception_message = fmt::format("Cannot extract table structure from {} format file, file is empty", format_name);
|
||||
|
||||
if (!retry)
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message);
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE, "{}. You can specify the structure manually", exception_message);
|
||||
|
||||
exception_messages += "\n" + exception_message;
|
||||
continue;
|
||||
@ -132,7 +134,8 @@ ColumnsDescription readSchemaFromFormat(
|
||||
max_rows_to_read -= schema_reader->getNumRowsRead();
|
||||
if (rows_read != 0 && max_rows_to_read == 0)
|
||||
{
|
||||
exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting input_format_max_rows_to_read_for_schema_inference";
|
||||
exception_message += "\nTo increase the maximum number of rows to read for structure determination, use setting "
|
||||
"input_format_max_rows_to_read_for_schema_inference";
|
||||
if (iterations > 1)
|
||||
{
|
||||
exception_messages += "\n" + exception_message;
|
||||
@ -150,15 +153,18 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
catch (Exception & e)
|
||||
{
|
||||
e.addMessage(fmt::format("Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
e.addMessage(fmt::format(
|
||||
"Cannot extract table structure from {} format file. You can specify the structure manually", format_name));
|
||||
throw;
|
||||
}
|
||||
catch (...)
|
||||
{
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
||||
"Cannot extract table structure from {} format file. "
|
||||
"Error: {}. You can specify the structure manually",
|
||||
format_name, exception_message);
|
||||
throw Exception(
|
||||
ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
||||
"Cannot extract table structure from {} format file. "
|
||||
"Error: {}. You can specify the structure manually",
|
||||
format_name,
|
||||
exception_message);
|
||||
}
|
||||
}
|
||||
|
||||
@ -170,9 +176,11 @@ ColumnsDescription readSchemaFromFormat(
|
||||
return cached_columns;
|
||||
|
||||
if (names_and_types.empty())
|
||||
throw Exception(ErrorCodes::CANNOT_EXTRACT_TABLE_STRUCTURE,
|
||||
"All attempts to extract table structure from files failed. "
|
||||
"Errors:{}\nYou can specify the structure manually", 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",
|
||||
exception_messages);
|
||||
|
||||
/// If we have "INSERT SELECT" query then try to order
|
||||
/// columns as they are ordered in table schema for formats
|
||||
@ -191,20 +199,30 @@ ColumnsDescription readSchemaFromFormat(
|
||||
}
|
||||
}
|
||||
else
|
||||
throw Exception(ErrorCodes::BAD_ARGUMENTS,
|
||||
"{} file format doesn't support schema inference. You must specify the structure manually",
|
||||
format_name);
|
||||
|
||||
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());
|
||||
return ColumnsDescription(names_and_types);
|
||||
}
|
||||
|
||||
ColumnsDescription readSchemaFromFormat(const String & format_name, const std::optional<FormatSettings> & format_settings, ReadBufferIterator & read_buffer_iterator, bool retry, ContextPtr & context)
|
||||
ColumnsDescription readSchemaFromFormat(
|
||||
const String & format_name,
|
||||
const std::optional<FormatSettings> & format_settings,
|
||||
ReadBufferIterator & read_buffer_iterator,
|
||||
bool retry,
|
||||
ContextPtr & context)
|
||||
{
|
||||
std::unique_ptr<ReadBuffer> buf_out;
|
||||
return readSchemaFromFormat(format_name, format_settings, read_buffer_iterator, retry, context, buf_out);
|
||||
}
|
||||
|
||||
SchemaCache::Key getKeyForSchemaCache(const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context)
|
||||
SchemaCache::Key getKeyForSchemaCache(
|
||||
const String & source, const String & format, const std::optional<FormatSettings> & format_settings, const ContextPtr & context)
|
||||
{
|
||||
return getKeysForSchemaCache({source}, format, format_settings, context).front();
|
||||
}
|
||||
@ -214,7 +232,8 @@ static SchemaCache::Key makeSchemaCacheKey(const String & source, const String &
|
||||
return SchemaCache::Key{source, format, additional_format_info};
|
||||
}
|
||||
|
||||
SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, 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)
|
||||
{
|
||||
/// For some formats data schema depends on some settings, so it's possible that
|
||||
/// two queries to the same source will get two different schemas. To process this
|
||||
@ -224,7 +243,11 @@ SchemaCache::Keys getKeysForSchemaCache(const Strings & sources, const String &
|
||||
String additional_format_info = FormatFactory::instance().getAdditionalInfoForSchemaCache(format, context, format_settings);
|
||||
SchemaCache::Keys cache_keys;
|
||||
cache_keys.reserve(sources.size());
|
||||
std::transform(sources.begin(), sources.end(), std::back_inserter(cache_keys), [&](const auto & source){ return makeSchemaCacheKey(source, format, additional_format_info); });
|
||||
std::transform(
|
||||
sources.begin(),
|
||||
sources.end(),
|
||||
std::back_inserter(cache_keys),
|
||||
[&](const auto & source) { return makeSchemaCacheKey(source, format, additional_format_info); });
|
||||
return cache_keys;
|
||||
}
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user