This commit is contained in:
Pavel Kruglov 2024-11-21 03:02:11 +01:00 committed by GitHub
commit 2678f62717
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
29 changed files with 351 additions and 68 deletions

View File

@ -1187,6 +1187,22 @@ namespace
source_ast->children.push_back(source_ast->elements);
dict.set(dict.source, source_ast);
}
ASTs * getEngineArgsFromCreateQuery(ASTCreateQuery & create_query)
{
ASTStorage * storage_def = create_query.storage;
if (!storage_def)
return nullptr;
if (!storage_def->engine)
return nullptr;
const ASTFunction & engine_def = *storage_def->engine;
if (!engine_def.arguments)
return nullptr;
return &engine_def.arguments->children;
}
}
void InterpreterCreateQuery::setEngine(ASTCreateQuery & create) const
@ -1876,7 +1892,11 @@ bool InterpreterCreateQuery::doCreateTable(ASTCreateQuery & create,
mode);
/// If schema wes inferred while storage creation, add columns description to create query.
addColumnsDescriptionToCreateQueryIfNecessary(query_ptr->as<ASTCreateQuery &>(), res);
auto & create_query = query_ptr->as<ASTCreateQuery &>();
addColumnsDescriptionToCreateQueryIfNecessary(create_query, res);
/// Add any inferred engine args if needed. For example, data format for engines File/S3/URL/etc
if (auto * engine_args = getEngineArgsFromCreateQuery(create_query))
res->addInferredEngineArgsToCreateQuery(*engine_args, getContext());
}
validateVirtualColumns(*res);

View File

@ -284,6 +284,10 @@ public:
/// Returns hints for serialization of columns accorsing to statistics accumulated by storage.
virtual SerializationInfoByName getSerializationHints() const { return {}; }
/// Add engine args that were inferred during storage creation to create query to avoid the same
/// inference on server restart. For example - data format inference in File/URL/S3/etc engines.
virtual void addInferredEngineArgsToCreateQuery(ASTs & /*args*/, const ContextPtr & /*context*/) const {}
private:
StorageID storage_id;

View File

@ -283,7 +283,7 @@ void StorageAzureConfiguration::fromAST(ASTs & engine_args, ContextPtr context,
}
void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure)
{
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
@ -295,7 +295,7 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
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")
if (with_structure && 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));
@ -319,9 +319,12 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
if (args.size() == 3)
{
args.push_back(format_literal);
/// Add compression = "auto" before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
if (with_structure)
{
/// 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)
@ -334,12 +337,15 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
{
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);
if (with_structure)
{
/// Add compression=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
}
}
/// (..., structure) -> (..., format, compression, structure)
else
else if (with_structure)
{
auto structure_arg = args.back();
args[3] = format_literal;
@ -362,15 +368,19 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
{
if (fourth_arg == "auto")
args[3] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// (..., 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);
if (with_structure)
{
/// 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
@ -386,7 +396,7 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
{
if (fourth_arg == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[5], "structure") == "auto")
args[5] = structure_literal;
}
/// (..., account_name, account_key, format) -> (..., account_name, account_key, format, compression, structure)
@ -394,12 +404,15 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
{
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);
if (with_structure)
{
/// 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
else if (with_structure)
{
auto structure_arg = args.back();
args[5] = format_literal;
@ -417,14 +430,15 @@ void StorageAzureConfiguration::addStructureAndFormatToArgsIfNeeded(
/// (..., format, compression) -> (..., format, compression, structure)
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// (storage_account_url, container_name, blobpath, account_name, account_key, format, compression, structure)
else if (args.size() == 8)
{
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = format_literal;
if (checkAndGetLiteralArgument<String>(args[7], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[7], "structure") == "auto")
args[7] = structure_literal;
}
}

View File

@ -76,7 +76,8 @@ public:
ASTs & args,
const String & structure_,
const String & format_,
ContextPtr context) override;
ContextPtr context,
bool with_structure) override;
protected:
void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;

View File

@ -175,7 +175,8 @@ void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded(
ASTs & args,
const String & structure_,
const String & format_,
ContextPtr context)
ContextPtr context,
bool with_structure)
{
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
@ -187,7 +188,7 @@ void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded(
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")
if (with_structure && 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));
@ -210,23 +211,26 @@ void StorageHDFSConfiguration::addStructureAndFormatToArgsIfNeeded(
if (count == 1)
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
args.push_back(format_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// hdfs(url, format)
else if (count == 2)
{
if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args.back() = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// hdfs(url, format, structure)
/// hdfs(url, format, structure, compression_method)
/// hdfs(url, format, compression_method)
else if (count >= 3)
{
if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
}

View File

@ -62,7 +62,8 @@ public:
ASTs & args,
const String & structure_,
const String & format_,
ContextPtr context) override;
ContextPtr context,
bool with_structure) override;
private:
void fromNamedCollection(const NamedCollection &, ContextPtr context) override;

View File

@ -59,7 +59,7 @@ public:
ObjectStoragePtr createObjectStorage(ContextPtr, bool) override { return std::make_shared<LocalObjectStorage>("/"); }
void addStructureAndFormatToArgsIfNeeded(ASTs &, const String &, const String &, ContextPtr) override { }
void addStructureAndFormatToArgsIfNeeded(ASTs &, const String &, const String &, ContextPtr, bool) override { }
private:
void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;

View File

@ -395,7 +395,7 @@ void StorageS3Configuration::fromAST(ASTs & args, ContextPtr context, bool with_
}
void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context)
ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure)
{
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
@ -407,7 +407,7 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
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")
if (with_structure && 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));
@ -429,8 +429,9 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
if (count == 1)
{
/// Add format=auto before structure argument.
args.push_back(std::make_shared<ASTLiteral>("auto"));
args.push_back(structure_literal);
args.push_back(format_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// s3(s3_url, format) or
/// s3(s3_url, NOSIGN)
@ -444,11 +445,13 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
else if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args[1] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
/// s3(source, format, structure) or
/// s3(source, access_key_id, secret_access_key) or
/// s3(source, NOSIGN, format)
/// s3(source, NOSIGN, format) or
/// s3(source, format, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither.
else if (count == 3)
{
@ -457,26 +460,29 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg))
{
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
else
{
/// Add format and structure arguments.
args.push_back(format_literal);
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
}
/// s3(source, format, structure, compression_method) or
/// s3(source, access_key_id, secret_access_key, format) or
/// s3(source, access_key_id, secret_access_key, session_token) or
/// s3(source, NOSIGN, format, structure)
/// s3(source, NOSIGN, format, structure) or
/// s3(source, NOSIGN, format, compression_method)
/// We can distinguish them by looking at the 2-nd argument: check if it's NOSIGN, format name or neither.
else if (count == 4)
{
@ -485,14 +491,14 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[3], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[3], "structure") == "auto")
args[3] = structure_literal;
}
else if (second_arg == "auto" || FormatFactory::instance().exists(second_arg))
{
if (second_arg == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
else
@ -502,18 +508,21 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
else
{
args.push_back(format_literal);
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
}
}
/// s3(source, access_key_id, secret_access_key, format, structure) or
/// s3(source, access_key_id, secret_access_key, session_token, format) or
/// s3(source, NOSIGN, format, structure, compression_method)
/// s3(source, NOSIGN, format, structure, compression_method) or
/// s3(source, access_key_id, secret_access_key, format, compression)
/// We can distinguish them by looking at the 2-nd argument: check if it's a NOSIGN keyword name or not.
else if (count == 5)
{
@ -522,7 +531,7 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[2], "format") == "auto")
args[2] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[3] = structure_literal;
}
else
@ -532,19 +541,21 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
else
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
args.push_back(structure_literal);
if (with_structure)
args.push_back(structure_literal);
}
}
}
/// s3(source, access_key_id, secret_access_key, format, structure, compression) or
/// s3(source, access_key_id, secret_access_key, session_token, format, structure)
/// s3(source, access_key_id, secret_access_key, session_token, format, structure) or
/// s3(source, access_key_id, secret_access_key, session_token, format, compression_method)
else if (count == 6)
{
auto fourth_arg = checkAndGetLiteralArgument<String>(args[3], "format/session_token");
@ -552,14 +563,14 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[3], "format") == "auto")
args[3] = format_literal;
if (checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[4], "structure") == "auto")
args[4] = structure_literal;
}
else
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = structure_literal;
}
}
@ -568,7 +579,7 @@ void StorageS3Configuration::addStructureAndFormatToArgsIfNeeded(
{
if (checkAndGetLiteralArgument<String>(args[4], "format") == "auto")
args[4] = format_literal;
if (checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[5], "format") == "auto")
args[5] = structure_literal;
}
}

View File

@ -91,7 +91,8 @@ public:
ASTs & args,
const String & structure,
const String & format,
ContextPtr context) override;
ContextPtr context,
bool with_structure) override;
private:
void fromNamedCollection(const NamedCollection & collection, ContextPtr context) override;

View File

@ -483,6 +483,11 @@ std::pair<ColumnsDescription, std::string> StorageObjectStorage::resolveSchemaAn
return std::pair(columns, format);
}
void StorageObjectStorage::addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const
{
configuration->addStructureAndFormatToArgsIfNeeded(args, "", configuration->format, context, /*with_structure=*/false);
}
SchemaCache & StorageObjectStorage::getSchemaCache(const ContextPtr & context, const std::string & storage_type_name)
{
if (storage_type_name == "s3")

View File

@ -122,6 +122,8 @@ public:
std::string & sample_path,
const ContextPtr & context);
void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override;
protected:
String getPathSample(StorageInMemoryMetadata metadata, ContextPtr context);
@ -179,7 +181,7 @@ public:
/// Add/replace structure and format arguments in the AST arguments if they have 'auto' values.
virtual void addStructureAndFormatToArgsIfNeeded(
ASTs & args, const String & structure_, const String & format_, ContextPtr context) = 0;
ASTs & args, const String & structure_, const String & format_, ContextPtr context, bool with_structure) = 0;
bool withPartitionWildcard() const;
bool withGlobs() const { return isPathWithGlobs() || isNamespaceWithGlobs(); }

View File

@ -107,7 +107,7 @@ void StorageObjectStorageCluster::updateQueryToSendIfNeeded(
ASTPtr cluster_name_arg = args.front();
args.erase(args.begin());
configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context);
configuration->addStructureAndFormatToArgsIfNeeded(args, structure, configuration->format, context, /*with_structure=*/true);
args.insert(args.begin(), cluster_name_arg);
}

View File

@ -2113,6 +2113,11 @@ void StorageFile::truncate(
}
}
void StorageFile::addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const
{
if (checkAndGetLiteralArgument<String>(evaluateConstantExpressionOrIdentifierAsLiteral(args[0], context), "format") == "auto")
args[0] = std::make_shared<ASTLiteral>(format_name);
}
void registerStorageFile(StorageFactory & factory)
{

View File

@ -139,6 +139,8 @@ public:
bool supportsTrivialCountOptimization(const StorageSnapshotPtr &, ContextPtr) const override { return true; }
void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override;
protected:
friend class StorageFileSource;
friend class StorageFileSink;

View File

@ -38,6 +38,8 @@
#include <Common/logger_useful.h>
#include <Common/re2.h>
#include <TableFunctions/TableFunctionURL.h>
#include <Formats/SchemaInferenceUtils.h>
#include <Core/FormatFactorySettings.h>
#include <Core/ServerSettings.h>
@ -1570,6 +1572,11 @@ void StorageURL::processNamedCollectionResult(Configuration & configuration, con
configuration.structure = collection.getOrDefault<String>("structure", "auto");
}
void StorageURL::addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const
{
TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(args, "", format_name, context, /*with_structure=*/false);
}
StorageURL::Configuration StorageURL::getConfiguration(ASTs & args, const ContextPtr & local_context)
{
StorageURL::Configuration configuration;

View File

@ -305,6 +305,8 @@ public:
bool supportsDynamicSubcolumns() const override { return true; }
void addInferredEngineArgsToCreateQuery(ASTs & args, const ContextPtr & context) const override;
static FormatSettings getFormatSettingsFromArgs(const StorageFactory::Arguments & args);
struct Configuration : public StatelessTableEngineConfiguration

View File

@ -31,7 +31,7 @@ public:
ASTPtr cluster_name_arg = args.front();
args.erase(args.begin());
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
Base::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context, /*with_structure=*/true);
args.insert(args.begin(), cluster_name_arg);
}

View File

@ -88,7 +88,7 @@ void ITableFunctionFileLike::parseArgumentsImpl(ASTs & args, const ContextPtr &
compression_method = checkAndGetLiteralArgument<String>(args[3], "compression_method");
}
void ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context)
void ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr & context, bool with_structure)
{
if (args.empty() || args.size() > getMaxNumberOfArguments())
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected 1 to {} arguments in table function, got {}", getMaxNumberOfArguments(), args.size());
@ -103,21 +103,23 @@ void ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(ASTs & ar
if (args.size() == 1)
{
args.push_back(format_literal);
args.push_back(structure_literal);
if (with_structure)
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);
if (with_structure)
args.push_back(structure_literal);
}
/// f(filename, format, structure) or f(filename, format, structure, compression)
/// f(filename, format, structure) or f(filename, format, structure, compression) or f(filename, format, compression)
else if (args.size() >= 3)
{
if (checkAndGetLiteralArgument<String>(args[1], "format") == "auto")
args[1] = format_literal;
if (checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
if (with_structure && checkAndGetLiteralArgument<String>(args[2], "structure") == "auto")
args[2] = structure_literal;
}
}

View File

@ -35,7 +35,7 @@ public:
static size_t getMaxNumberOfArguments() { return max_number_of_arguments; }
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure, const String & format, const ContextPtr &, bool with_structure);
protected:

View File

@ -139,7 +139,7 @@ public:
const String & format,
const ContextPtr & context)
{
Configuration().addStructureAndFormatToArgsIfNeeded(args, structure, format, context);
Configuration().addStructureAndFormatToArgsIfNeeded(args, structure, format, context, /*with_structure=*/true);
}
protected:

View File

@ -77,7 +77,7 @@ void TableFunctionURL::parseArgumentsImpl(ASTs & args, const ContextPtr & contex
}
}
void TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context)
void TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context, bool with_structure)
{
if (auto collection = tryGetNamedCollectionWithOverrides(args, context))
{
@ -89,7 +89,7 @@ void TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, co
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")
if (with_structure && 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));
@ -109,7 +109,7 @@ void TableFunctionURL::updateStructureAndFormatArgumentsIfNeeded(ASTs & args, co
args.pop_back();
}
ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context);
ITableFunctionFileLike::updateStructureAndFormatArgumentsIfNeeded(args, structure_, format_, context, with_structure);
if (headers_ast)
args.push_back(headers_ast);

View File

@ -34,7 +34,7 @@ public:
ColumnsDescription getActualTableStructure(ContextPtr context, bool is_insert_query) const override;
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context);
static void updateStructureAndFormatArgumentsIfNeeded(ASTs & args, const String & structure_, const String & format_, const ContextPtr & context, bool with_structure);
protected:
void parseArguments(const ASTPtr & ast, ContextPtr context) override;

View File

@ -1314,6 +1314,7 @@ def test_size_virtual_column(cluster):
def test_format_detection(cluster):
node = cluster.instances["node"]
connection_string = cluster.env_variables["AZURITE_CONNECTION_STRING"]
storage_account_url = cluster.env_variables["AZURITE_STORAGE_ACCOUNT_URL"]
account_name = "devstoreaccount1"
account_key = "Eby8vdM02xNOcqFlqUwJPLlmEtlCDXJ1OUzFT50uSRZ6IFsuFq2UVErCz4I6tq/K1SZFPTOtr/KBHBeksoGMGw=="
@ -1381,6 +1382,84 @@ def test_format_detection(cluster):
assert result == expected_result
azure_query(
node,
f"create table test_format_detection engine=AzureBlobStorage('{connection_string}', 'cont', 'test_format_detection1')",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{connection_string}\\', \\'cont\\', \\'test_format_detection1\\', \\'JSON\\')\n"
)
azure_query(
node,
f"create or replace table test_format_detection engine=AzureBlobStorage('{connection_string}', 'cont', 'test_format_detection1', auto)",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{connection_string}\\', \\'cont\\', \\'test_format_detection1\\', \\'JSON\\')\n"
)
azure_query(
node,
f"create or replace table test_format_detection engine=AzureBlobStorage('{connection_string}', 'cont', 'test_format_detection1', auto, 'none')",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{connection_string}\\', \\'cont\\', \\'test_format_detection1\\', \\'JSON\\', \\'none\\')\n"
)
azure_query(
node,
f"create or replace table test_format_detection engine=AzureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}')",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{storage_account_url}\\', \\'cont\\', \\'test_format_detection1\\', \\'{account_name}\\', \\'{account_key}\\', \\'JSON\\')\n"
)
azure_query(
node,
f"create or replace table test_format_detection engine=AzureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', auto)",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{storage_account_url}\\', \\'cont\\', \\'test_format_detection1\\', \\'{account_name}\\', \\'{account_key}\\', \\'JSON\\')\n"
)
azure_query(
node,
f"create or replace table test_format_detection engine=AzureBlobStorage('{storage_account_url}', 'cont', 'test_format_detection1', '{account_name}', '{account_key}', auto, 'none')",
)
result = azure_query(
node,
f"show create table test_format_detection",
)
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = AzureBlobStorage(\\'{storage_account_url}\\', \\'cont\\', \\'test_format_detection1\\', \\'{account_name}\\', \\'{account_key}\\', \\'JSON\\', \\'none\\')\n"
)
def test_write_to_globbed_partitioned_path(cluster):
node = cluster.instances["node"]

View File

@ -636,7 +636,7 @@ def test_multiple_inserts(started_cluster):
node1.query(f"drop table test_multiple_inserts")
def test_format_detection(started_cluster):
def test_format_detection_from_file_name(started_cluster):
node1.query(
f"create table arrow_table (x UInt64) engine=HDFS('hdfs://hdfs1:9000/data.arrow')"
)
@ -1222,6 +1222,35 @@ def test_format_detection(started_cluster):
assert expected_result == result
node.query(
f"create table test_format_detection engine=HDFS('hdfs://hdfs1:9000/{dir}/test_format_detection1')"
)
result = node.query(f"show create table test_format_detection")
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = HDFS(\\'hdfs://hdfs1:9000/{dir}/test_format_detection1\\', \\'JSON\\')\n"
)
node.query("drop table test_format_detection")
node.query(
f"create table test_format_detection engine=HDFS('hdfs://hdfs1:9000/{dir}/test_format_detection1', auto)"
)
result = node.query(f"show create table test_format_detection")
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = HDFS(\\'hdfs://hdfs1:9000/{dir}/test_format_detection1\\', \\'JSON\\')\n"
)
node.query("drop table test_format_detection")
node.query(
f"create table test_format_detection engine=HDFS('hdfs://hdfs1:9000/{dir}/test_format_detection1', auto, 'none')"
)
result = node.query(f"show create table test_format_detection")
assert (
result
== f"CREATE TABLE default.test_format_detection\\n(\\n `x` Nullable(String),\\n `y` Nullable(String)\\n)\\nENGINE = HDFS(\\'hdfs://hdfs1:9000/{dir}/test_format_detection1\\', \\'JSON\\', \\'none\\')\n"
)
def test_write_to_globbed_partitioned_path(started_cluster):
node = started_cluster.instances["node1"]

View File

@ -0,0 +1,14 @@
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'NOSIGN\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'JSON\', \'none\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'NOSIGN\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'test\', \'[HIDDEN]\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'NOSIGN\', \'JSON\', \'none\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'test\', \'[HIDDEN]\', \'\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'test\', \'[HIDDEN]\', \'\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'test\', \'[HIDDEN]\', \'JSON\', \'none\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = S3(\'http://localhost:11111/test/json_data\', \'test\', \'[HIDDEN]\', \'\', \'JSON\', \'none\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = URL(\'http://localhost:11111/test/json_data\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = URL(\'http://localhost:11111/test/json_data\', \'JSON\')
CREATE TABLE default.test\n(\n `a` Nullable(Int64)\n)\nENGINE = URL(\'http://localhost:11111/test/json_data\', \'JSON\', \'none\')

View File

@ -0,0 +1,59 @@
-- Tags: no-fasttest
drop table if exists test;
create table test engine=S3('http://localhost:11111/test/json_data');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', NOSIGN);
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', auto);
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', auto, 'none');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', NOSIGN, auto);
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', 'test', 'testtest');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', NOSIGN, auto, 'none');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', 'test', 'testtest', '');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', 'test', 'testtest', '', auto);
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', 'test', 'testtest', auto, 'none');
show create table test;
drop table test;
create table test engine=S3('http://localhost:11111/test/json_data', 'test', 'testtest', '', auto, 'none');
show create table test;
drop table test;
create table test engine=URL('http://localhost:11111/test/json_data');
show create table test;
drop table test;
create table test engine=URL('http://localhost:11111/test/json_data', auto);
show create table test;
drop table test;
create table test engine=URL('http://localhost:11111/test/json_data', auto, 'none');
show create table test;
drop table test;

View File

@ -0,0 +1,19 @@
#!/usr/bin/env bash
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CURDIR"/../shell_config.sh
$CLICKHOUSE_LOCAL -q "select 42 as a format JSONEachRow" > data_$CLICKHOUSE_TEST_UNIQUE_NAME
$CLICKHOUSE_LOCAL -nm -q "
create table test engine=File(auto, './data_$CLICKHOUSE_TEST_UNIQUE_NAME');
show create table test;
drop table test;
create table test engine=File(auto, './data_$CLICKHOUSE_TEST_UNIQUE_NAME', 'none');
show create table test;
drop table test;
" | grep "JSON" -c
rm data_$CLICKHOUSE_TEST_UNIQUE_NAME

View File

@ -0,0 +1 @@
{"a" : 42}