rename to s3Cluster

This commit is contained in:
Nikita Mikhaylov 2021-04-13 00:42:52 +03:00
parent a743442a17
commit 09a62e713a
8 changed files with 28 additions and 28 deletions

View File

@ -25,7 +25,7 @@
#include <Interpreters/OpenTelemetrySpanLog.h>
#include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/StorageS3Distributed.h>
#include <Storages/StorageS3Cluster.h>
#include <Core/ExternalTable.h>
#include <Storages/ColumnDefault.h>
#include <DataTypes/DataTypeLowCardinality.h>

View File

@ -128,8 +128,8 @@ public:
private:
friend class StorageS3Distributed;
friend class TableFunctionS3Distributed;
friend class StorageS3Cluster;
friend class TableFunctionS3Cluster;
friend class StorageS3SequentialSource;
struct ClientAuthentificaiton

View File

@ -1,4 +1,4 @@
#include "Storages/StorageS3Distributed.h"
#include "Storages/StorageS3Cluster.h"
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
@ -53,7 +53,7 @@ namespace DB
{
StorageS3Distributed::StorageS3Distributed(
StorageS3Cluster::StorageS3Cluster(
const String & filename_,
const String & access_key_id_,
const String & secret_access_key_,
@ -81,7 +81,7 @@ StorageS3Distributed::StorageS3Distributed(
}
Pipe StorageS3Distributed::read(
Pipe StorageS3Cluster::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info,
@ -149,7 +149,7 @@ Pipe StorageS3Distributed::read(
connections.emplace_back(std::make_shared<Connection>(
node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(),
node.user, node.password, node.cluster, node.cluster_secret,
"S3DistributedInititiator",
"S3ClusterInititiator",
node.compression,
node.secure
));
@ -168,7 +168,7 @@ Pipe StorageS3Distributed::read(
return Pipe::unitePipes(std::move(pipes));
}
QueryProcessingStage::Enum StorageS3Distributed::getQueryProcessingStage(
QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage(
ContextPtr context, QueryProcessingStage::Enum to_stage, SelectQueryInfo &) const
{
/// Initiator executes query on remote node.
@ -181,7 +181,7 @@ QueryProcessingStage::Enum StorageS3Distributed::getQueryProcessingStage(
}
NamesAndTypesList StorageS3Distributed::getVirtuals() const
NamesAndTypesList StorageS3Cluster::getVirtuals() const
{
return NamesAndTypesList{
{"_path", std::make_shared<DataTypeString>()},

View File

@ -27,11 +27,11 @@ struct ClientAuthentificationBuilder
UInt64 max_connections;
};
class StorageS3Distributed : public ext::shared_ptr_helper<StorageS3Distributed>, public IStorage
class StorageS3Cluster : public ext::shared_ptr_helper<StorageS3Cluster>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageS3Distributed>;
friend struct ext::shared_ptr_helper<StorageS3Cluster>;
public:
std::string getName() const override { return "S3Distributed"; }
std::string getName() const override { return "S3Cluster"; }
Pipe read(const Names &, const StorageMetadataPtr &, SelectQueryInfo &,
ContextPtr, QueryProcessingStage::Enum, size_t /*max_block_size*/, unsigned /*num_streams*/) override;
@ -41,7 +41,7 @@ public:
NamesAndTypesList getVirtuals() const override;
protected:
StorageS3Distributed(
StorageS3Cluster(
const String & filename_, const String & access_key_id_, const String & secret_access_key_, const StorageID & table_id_,
String cluster_name_, const String & format_name_, UInt64 max_connections_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, ContextPtr context_, const String & compression_method_);

View File

@ -4,7 +4,7 @@
#if USE_AWS_S3
#include <Storages/StorageS3Distributed.h>
#include <Storages/StorageS3Cluster.h>
#include <DataTypes/DataTypeString.h>
#include <DataStreams/RemoteBlockInputStream.h>
@ -15,7 +15,7 @@
#include <Interpreters/ClientInfo.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionS3.h>
#include <TableFunctions/TableFunctionS3Distributed.h>
#include <TableFunctions/TableFunctionS3Cluster.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
@ -37,7 +37,7 @@ namespace ErrorCodes
}
void TableFunctionS3Distributed::parseArguments(const ASTPtr & ast_function, ContextPtr context)
void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
/// Parse args
ASTs & args_func = ast_function->children;
@ -95,16 +95,16 @@ void TableFunctionS3Distributed::parseArguments(const ASTPtr & ast_function, Con
}
ColumnsDescription TableFunctionS3Distributed::getActualTableStructure(ContextPtr context) const
ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const
{
return parseColumnsListFromString(structure, context);
}
StoragePtr TableFunctionS3Distributed::executeImpl(
StoragePtr TableFunctionS3Cluster::executeImpl(
const ASTPtr & /*function*/, ContextPtr context,
const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
StoragePtr storage = StorageS3Distributed::create(
StoragePtr storage = StorageS3Cluster::create(
filename, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name),
cluster_name, format, context->getSettingsRef().s3_max_connections,
getActualTableStructure(context), ConstraintsDescription{},
@ -116,9 +116,9 @@ StoragePtr TableFunctionS3Distributed::executeImpl(
}
void registerTableFunctionS3Distributed(TableFunctionFactory & factory)
void registerTableFunctionS3Cluster(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionS3Distributed>();
factory.registerFunction<TableFunctionS3Cluster>();
}
void registerTableFunctionCOSDistributed(TableFunctionFactory & factory)

View File

@ -13,7 +13,7 @@ namespace DB
class Context;
/**
* s3Distributed(cluster_name, source, [access_key_id, secret_access_key,] format, structure)
* s3Cluster(cluster_name, source, [access_key_id, secret_access_key,] format, structure)
* A table function, which allows to process many files from S3 on a specific cluster
* On initiator it creates a connection to _all_ nodes in cluster, discloses asterics
* in S3 file path and register all tasks (paths in S3) in NextTaskResolver to dispatch
@ -21,10 +21,10 @@ class Context;
* On worker node it asks initiator about next task to process, processes it.
* This is repeated until the tasks are finished.
*/
class TableFunctionS3Distributed : public ITableFunction
class TableFunctionS3Cluster : public ITableFunction
{
public:
static constexpr auto name = "s3Distributed";
static constexpr auto name = "s3Cluster";
std::string getName() const override
{
return name;
@ -38,7 +38,7 @@ protected:
const std::string & table_name,
ColumnsDescription cached_columns) const override;
const char * getStorageTypeName() const override { return "S3Distributed"; }
const char * getStorageTypeName() const override { return "S3Cluster"; }
ColumnsDescription getActualTableStructure(ContextPtr) const override;
void parseArguments(const ASTPtr &, ContextPtr) override;
@ -52,7 +52,7 @@ protected:
String compression_method = "auto";
};
class TableFunctionCOSDistributed : public TableFunctionS3Distributed
class TableFunctionCOSDistributed : public TableFunctionS3Cluster
{
public:
static constexpr auto name = "cosnDistributed";

View File

@ -21,7 +21,7 @@ void registerTableFunctions()
#if USE_AWS_S3
registerTableFunctionS3(factory);
registerTableFunctionS3Distributed(factory);
registerTableFunctionS3Cluster(factory);
registerTableFunctionCOS(factory);
#endif

View File

@ -21,7 +21,7 @@ void registerTableFunctionGenerate(TableFunctionFactory & factory);
#if USE_AWS_S3
void registerTableFunctionS3(TableFunctionFactory & factory);
void registerTableFunctionS3Distributed(TableFunctionFactory & factory);
void registerTableFunctionS3Cluster(TableFunctionFactory & factory);
void registerTableFunctionCOS(TableFunctionFactory & factory);
#endif