Merge pull request #22012 from nikitamikhaylov/s3-distributed

Table function `s3Cluster`
This commit is contained in:
Nikita Mikhaylov 2021-04-14 10:23:43 +03:00 committed by GitHub
commit c0ed151239
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
46 changed files with 1243 additions and 321 deletions

View File

@ -366,6 +366,9 @@ function run_tests
# JSON functions # JSON functions
01666_blns 01666_blns
# Depends on AWS
01801_s3_cluster
) )
(time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt" (time clickhouse-test --hung-check -j 8 --order=random --use-skip-list --no-long --testname --shard --zookeeper --skip "${TESTS_TO_SKIP[@]}" -- "$FASTTEST_FOCUS" 2>&1 ||:) | ts '%Y-%m-%d %H:%M:%S' | tee "$FASTTEST_OUTPUT/test_log.txt"

View File

@ -551,6 +551,15 @@ void Connection::sendIgnoredPartUUIDs(const std::vector<UUID> & uuids)
out->next(); out->next();
} }
void Connection::sendReadTaskResponse(const String & response)
{
writeVarUInt(Protocol::Client::ReadTaskResponse, *out);
writeVarUInt(DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION, *out);
writeStringBinary(response, *out);
out->next();
}
void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name) void Connection::sendPreparedData(ReadBuffer & input, size_t size, const String & name)
{ {
/// NOTE 'Throttler' is not used in this method (could use, but it's not important right now). /// NOTE 'Throttler' is not used in this method (could use, but it's not important right now).
@ -807,6 +816,9 @@ Packet Connection::receivePacket()
readVectorBinary(res.part_uuids, *in); readVectorBinary(res.part_uuids, *in);
return res; return res;
case Protocol::Server::ReadTaskRequest:
return res;
default: default:
/// In unknown state, disconnect - to not leave unsynchronised connection. /// In unknown state, disconnect - to not leave unsynchronised connection.
disconnect(); disconnect();
@ -907,13 +919,13 @@ void Connection::setDescription()
} }
std::unique_ptr<Exception> Connection::receiveException() std::unique_ptr<Exception> Connection::receiveException() const
{ {
return std::make_unique<Exception>(readException(*in, "Received from " + getDescription(), true /* remote */)); return std::make_unique<Exception>(readException(*in, "Received from " + getDescription(), true /* remote */));
} }
std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type) std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type) const
{ {
size_t num = Protocol::Server::stringsInMessage(msg_type); size_t num = Protocol::Server::stringsInMessage(msg_type);
std::vector<String> strings(num); std::vector<String> strings(num);
@ -923,7 +935,7 @@ std::vector<String> Connection::receiveMultistringMessage(UInt64 msg_type)
} }
Progress Connection::receiveProgress() Progress Connection::receiveProgress() const
{ {
Progress progress; Progress progress;
progress.read(*in, server_revision); progress.read(*in, server_revision);
@ -931,7 +943,7 @@ Progress Connection::receiveProgress()
} }
BlockStreamProfileInfo Connection::receiveProfileInfo() BlockStreamProfileInfo Connection::receiveProfileInfo() const
{ {
BlockStreamProfileInfo profile_info; BlockStreamProfileInfo profile_info;
profile_info.read(*in); profile_info.read(*in);

View File

@ -159,6 +159,8 @@ public:
/// Send parts' uuids to excluded them from query processing /// Send parts' uuids to excluded them from query processing
void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids); void sendIgnoredPartUUIDs(const std::vector<UUID> & uuids);
void sendReadTaskResponse(const String &);
/// Send prepared block of data (serialized and, if need, compressed), that will be read from 'input'. /// Send prepared block of data (serialized and, if need, compressed), that will be read from 'input'.
/// You could pass size of serialized/compressed block. /// You could pass size of serialized/compressed block.
void sendPreparedData(ReadBuffer & input, size_t size, const String & name = ""); void sendPreparedData(ReadBuffer & input, size_t size, const String & name = "");
@ -269,7 +271,7 @@ private:
class LoggerWrapper class LoggerWrapper
{ {
public: public:
LoggerWrapper(Connection & parent_) explicit LoggerWrapper(Connection & parent_)
: log(nullptr), parent(parent_) : log(nullptr), parent(parent_)
{ {
} }
@ -304,10 +306,10 @@ private:
Block receiveLogData(); Block receiveLogData();
Block receiveDataImpl(BlockInputStreamPtr & stream); Block receiveDataImpl(BlockInputStreamPtr & stream);
std::vector<String> receiveMultistringMessage(UInt64 msg_type); std::vector<String> receiveMultistringMessage(UInt64 msg_type) const;
std::unique_ptr<Exception> receiveException(); std::unique_ptr<Exception> receiveException() const;
Progress receiveProgress(); Progress receiveProgress() const;
BlockStreamProfileInfo receiveProfileInfo(); BlockStreamProfileInfo receiveProfileInfo() const;
void initInputBuffers(); void initInputBuffers();
void initBlockInput(); void initBlockInput();

View File

@ -26,7 +26,7 @@ public:
using Entry = PoolBase<Connection>::Entry; using Entry = PoolBase<Connection>::Entry;
public: public:
virtual ~IConnectionPool() {} virtual ~IConnectionPool() = default;
/// Selects the connection to work. /// Selects the connection to work.
/// If force_connected is false, the client must manually ensure that returned connection is good. /// If force_connected is false, the client must manually ensure that returned connection is good.

View File

@ -14,6 +14,12 @@
namespace DB namespace DB
{ {
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
/** To receive data from multiple replicas (connections) from one shard asynchronously. /** To receive data from multiple replicas (connections) from one shard asynchronously.
* The principe of Hedged Connections is used to reduce tail latency: * The principe of Hedged Connections is used to reduce tail latency:
* if we don't receive data from replica and there is no progress in query execution * if we don't receive data from replica and there is no progress in query execution
@ -84,6 +90,11 @@ public:
const ClientInfo & client_info, const ClientInfo & client_info,
bool with_pending_data) override; bool with_pending_data) override;
void sendReadTaskResponse(const String &) override
{
throw Exception("sendReadTaskResponse in not supported with HedgedConnections", ErrorCodes::LOGICAL_ERROR);
}
Packet receivePacket() override; Packet receivePacket() override;
Packet receivePacketUnlocked(AsyncCallback async_callback) override; Packet receivePacketUnlocked(AsyncCallback async_callback) override;

View File

@ -24,6 +24,8 @@ public:
const ClientInfo & client_info, const ClientInfo & client_info,
bool with_pending_data) = 0; bool with_pending_data) = 0;
virtual void sendReadTaskResponse(const String &) = 0;
/// Get packet from any replica. /// Get packet from any replica.
virtual Packet receivePacket() = 0; virtual Packet receivePacket() = 0;

View File

@ -155,6 +155,15 @@ void MultiplexedConnections::sendIgnoredPartUUIDs(const std::vector<UUID> & uuid
} }
} }
void MultiplexedConnections::sendReadTaskResponse(const String & response)
{
std::lock_guard lock(cancel_mutex);
if (cancelled)
return;
current_connection->sendReadTaskResponse(response);
}
Packet MultiplexedConnections::receivePacket() Packet MultiplexedConnections::receivePacket()
{ {
std::lock_guard lock(cancel_mutex); std::lock_guard lock(cancel_mutex);
@ -210,6 +219,7 @@ Packet MultiplexedConnections::drain()
switch (packet.type) switch (packet.type)
{ {
case Protocol::Server::ReadTaskRequest:
case Protocol::Server::PartUUIDs: case Protocol::Server::PartUUIDs:
case Protocol::Server::Data: case Protocol::Server::Data:
case Protocol::Server::Progress: case Protocol::Server::Progress:
@ -273,6 +283,7 @@ Packet MultiplexedConnections::receivePacketUnlocked(AsyncCallback async_callbac
switch (packet.type) switch (packet.type)
{ {
case Protocol::Server::ReadTaskRequest:
case Protocol::Server::PartUUIDs: case Protocol::Server::PartUUIDs:
case Protocol::Server::Data: case Protocol::Server::Data:
case Protocol::Server::Progress: case Protocol::Server::Progress:

View File

@ -39,6 +39,8 @@ public:
const ClientInfo & client_info, const ClientInfo & client_info,
bool with_pending_data) override; bool with_pending_data) override;
void sendReadTaskResponse(const String &) override;
Packet receivePacket() override; Packet receivePacket() override;
void disconnect() override; void disconnect() override;

View File

@ -51,7 +51,7 @@ private:
*/ */
struct PoolEntryHelper struct PoolEntryHelper
{ {
PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; } explicit PoolEntryHelper(PooledObject & data_) : data(data_) { data.in_use = true; }
~PoolEntryHelper() ~PoolEntryHelper()
{ {
std::unique_lock lock(data.pool.mutex); std::unique_lock lock(data.pool.mutex);
@ -69,7 +69,7 @@ public:
public: public:
friend class PoolBase<Object>; friend class PoolBase<Object>;
Entry() {} /// For deferred initialization. Entry() = default; /// For deferred initialization.
/** The `Entry` object protects the resource from being used by another thread. /** The `Entry` object protects the resource from being used by another thread.
* The following methods are forbidden for `rvalue`, so you can not write a similar to * The following methods are forbidden for `rvalue`, so you can not write a similar to
@ -99,10 +99,10 @@ public:
private: private:
std::shared_ptr<PoolEntryHelper> data; std::shared_ptr<PoolEntryHelper> data;
Entry(PooledObject & object) : data(std::make_shared<PoolEntryHelper>(object)) {} explicit Entry(PooledObject & object) : data(std::make_shared<PoolEntryHelper>(object)) {}
}; };
virtual ~PoolBase() {} virtual ~PoolBase() = default;
/** Allocates the object. Wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. */ /** Allocates the object. Wait for free object in pool for 'timeout'. With 'timeout' < 0, the timeout is infinite. */
Entry get(Poco::Timespan::TimeDiff timeout) Entry get(Poco::Timespan::TimeDiff timeout)

View File

@ -74,6 +74,9 @@
/// Minimum revision supporting OpenTelemetry /// Minimum revision supporting OpenTelemetry
#define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442 #define DBMS_MIN_REVISION_WITH_OPENTELEMETRY 54442
#define DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION 1
/// Minimum revision supporting interserver secret. /// Minimum revision supporting interserver secret.
#define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441 #define DBMS_MIN_REVISION_WITH_INTERSERVER_SECRET 54441

View File

@ -76,8 +76,10 @@ namespace Protocol
Log = 10, /// System logs of the query execution Log = 10, /// System logs of the query execution
TableColumns = 11, /// Columns' description for default values calculation TableColumns = 11, /// Columns' description for default values calculation
PartUUIDs = 12, /// List of unique parts ids. PartUUIDs = 12, /// List of unique parts ids.
ReadTaskRequest = 13, /// String (UUID) describes a request for which next task is needed
MAX = PartUUIDs, /// This is such an inverted logic, where server sends requests
/// And client returns back response
MAX = ReadTaskRequest,
}; };
/// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10 /// NOTE: If the type of packet argument would be Enum, the comparison packet >= 0 && packet < 10
@ -100,6 +102,7 @@ namespace Protocol
"Log", "Log",
"TableColumns", "TableColumns",
"PartUUIDs", "PartUUIDs",
"ReadTaskRequest"
}; };
return packet <= MAX return packet <= MAX
? data[packet] ? data[packet]
@ -135,8 +138,9 @@ namespace Protocol
KeepAlive = 6, /// Keep the connection alive KeepAlive = 6, /// Keep the connection alive
Scalar = 7, /// A block of data (compressed or not). Scalar = 7, /// A block of data (compressed or not).
IgnoredPartUUIDs = 8, /// List of unique parts ids to exclude from query processing IgnoredPartUUIDs = 8, /// List of unique parts ids to exclude from query processing
ReadTaskResponse = 9, /// TODO:
MAX = IgnoredPartUUIDs, MAX = ReadTaskResponse,
}; };
inline const char * toString(UInt64 packet) inline const char * toString(UInt64 packet)
@ -151,6 +155,7 @@ namespace Protocol
"KeepAlive", "KeepAlive",
"Scalar", "Scalar",
"IgnoredPartUUIDs", "IgnoredPartUUIDs",
"ReadTaskResponse",
}; };
return packet <= MAX return packet <= MAX
? data[packet] ? data[packet]

View File

@ -22,20 +22,18 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR;
extern const int UNKNOWN_PACKET_FROM_SERVER; extern const int UNKNOWN_PACKET_FROM_SERVER;
extern const int DUPLICATED_PART_UUIDS; extern const int DUPLICATED_PART_UUIDS;
} }
RemoteQueryExecutor::RemoteQueryExecutor( RemoteQueryExecutor::RemoteQueryExecutor(
Connection & connection, Connection & connection,
const String & query_, const String & query_, const Block & header_, ContextPtr context_,
const Block & header_, ThrottlerPtr throttler, const Scalars & scalars_, const Tables & external_tables_,
ContextPtr context_, QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
ThrottlerPtr throttler, : header(header_), query(query_), context(context_)
const Scalars & scalars_, , scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
const Tables & external_tables_,
QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{ {
create_connections = [this, &connection, throttler]() create_connections = [this, &connection, throttler]()
{ {
@ -45,14 +43,11 @@ RemoteQueryExecutor::RemoteQueryExecutor(
RemoteQueryExecutor::RemoteQueryExecutor( RemoteQueryExecutor::RemoteQueryExecutor(
std::vector<IConnectionPool::Entry> && connections_, std::vector<IConnectionPool::Entry> && connections_,
const String & query_, const String & query_, const Block & header_, ContextPtr context_,
const Block & header_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_,
ContextPtr context_, QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
const ThrottlerPtr & throttler, : header(header_), query(query_), context(context_)
const Scalars & scalars_, , scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
const Tables & external_tables_,
QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{ {
create_connections = [this, connections_, throttler]() mutable { create_connections = [this, connections_, throttler]() mutable {
return std::make_unique<MultiplexedConnections>(std::move(connections_), context->getSettingsRef(), throttler); return std::make_unique<MultiplexedConnections>(std::move(connections_), context->getSettingsRef(), throttler);
@ -61,14 +56,11 @@ RemoteQueryExecutor::RemoteQueryExecutor(
RemoteQueryExecutor::RemoteQueryExecutor( RemoteQueryExecutor::RemoteQueryExecutor(
const ConnectionPoolWithFailoverPtr & pool, const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const String & query_, const Block & header_, ContextPtr context_,
const Block & header_, const ThrottlerPtr & throttler, const Scalars & scalars_, const Tables & external_tables_,
ContextPtr context_, QueryProcessingStage::Enum stage_, std::shared_ptr<TaskIterator> task_iterator_)
const ThrottlerPtr & throttler, : header(header_), query(query_), context(context_)
const Scalars & scalars_, , scalars(scalars_), external_tables(external_tables_), stage(stage_), task_iterator(task_iterator_)
const Tables & external_tables_,
QueryProcessingStage::Enum stage_)
: header(header_), query(query_), context(context_), scalars(scalars_), external_tables(external_tables_), stage(stage_)
{ {
create_connections = [this, pool, throttler]()->std::unique_ptr<IConnections> create_connections = [this, pool, throttler]()->std::unique_ptr<IConnections>
{ {
@ -307,6 +299,9 @@ std::optional<Block> RemoteQueryExecutor::processPacket(Packet packet)
{ {
switch (packet.type) switch (packet.type)
{ {
case Protocol::Server::ReadTaskRequest:
processReadTaskRequest();
break;
case Protocol::Server::PartUUIDs: case Protocol::Server::PartUUIDs:
if (!setPartUUIDs(packet.part_uuids)) if (!setPartUUIDs(packet.part_uuids))
got_duplicated_part_uuids = true; got_duplicated_part_uuids = true;
@ -385,6 +380,14 @@ bool RemoteQueryExecutor::setPartUUIDs(const std::vector<UUID> & uuids)
return true; return true;
} }
void RemoteQueryExecutor::processReadTaskRequest()
{
if (!task_iterator)
throw Exception("Distributed task iterator is not initialized", ErrorCodes::LOGICAL_ERROR);
auto response = (*task_iterator)();
connections->sendReadTaskResponse(response);
}
void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context) void RemoteQueryExecutor::finish(std::unique_ptr<ReadContext> * read_context)
{ {
/** If one of: /** If one of:

View File

@ -26,6 +26,9 @@ using ProfileInfoCallback = std::function<void(const BlockStreamProfileInfo & in
class RemoteQueryExecutorReadContext; class RemoteQueryExecutorReadContext;
/// This is the same type as StorageS3Source::IteratorWrapper
using TaskIterator = std::function<String()>;
/// This class allows one to launch queries on remote replicas of one shard and get results /// This class allows one to launch queries on remote replicas of one shard and get results
class RemoteQueryExecutor class RemoteQueryExecutor
{ {
@ -37,21 +40,21 @@ public:
Connection & connection, Connection & connection,
const String & query_, const Block & header_, ContextPtr context_, const String & query_, const Block & header_, ContextPtr context_,
ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), ThrottlerPtr throttler_ = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
/// Accepts several connections already taken from pool. /// Accepts several connections already taken from pool.
RemoteQueryExecutor( RemoteQueryExecutor(
std::vector<IConnectionPool::Entry> && connections_, std::vector<IConnectionPool::Entry> && connections_,
const String & query_, const Block & header_, ContextPtr context_, const String & query_, const Block & header_, ContextPtr context_,
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
/// Takes a pool and gets one or several connections from it. /// Takes a pool and gets one or several connections from it.
RemoteQueryExecutor( RemoteQueryExecutor(
const ConnectionPoolWithFailoverPtr & pool, const ConnectionPoolWithFailoverPtr & pool,
const String & query_, const Block & header_, ContextPtr context_, const String & query_, const Block & header_, ContextPtr context_,
const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(), const ThrottlerPtr & throttler = nullptr, const Scalars & scalars_ = Scalars(), const Tables & external_tables_ = Tables(),
QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete); QueryProcessingStage::Enum stage_ = QueryProcessingStage::Complete, std::shared_ptr<TaskIterator> task_iterator_ = {});
~RemoteQueryExecutor(); ~RemoteQueryExecutor();
@ -119,6 +122,8 @@ private:
/// Temporary tables needed to be sent to remote servers /// Temporary tables needed to be sent to remote servers
Tables external_tables; Tables external_tables;
QueryProcessingStage::Enum stage; QueryProcessingStage::Enum stage;
/// Initiator identifier for distributed task processing
std::shared_ptr<TaskIterator> task_iterator;
/// Streams for reading from temporary tables and following sending of data /// Streams for reading from temporary tables and following sending of data
/// to remote servers for GLOBAL-subqueries /// to remote servers for GLOBAL-subqueries
@ -179,6 +184,8 @@ private:
/// Return true if duplicates found. /// Return true if duplicates found.
bool setPartUUIDs(const std::vector<UUID> & uuids); bool setPartUUIDs(const std::vector<UUID> & uuids);
void processReadTaskRequest();
/// Cancell query and restart it with info about duplicated UUIDs /// Cancell query and restart it with info about duplicated UUIDs
/// only for `allow_experimental_query_deduplication`. /// only for `allow_experimental_query_deduplication`.
std::variant<Block, int> restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context = nullptr); std::variant<Block, int> restartQueryWithoutDuplicatedUUIDs(std::unique_ptr<ReadContext> * read_context = nullptr);

View File

@ -477,7 +477,7 @@ DataTypePtr FunctionOverloadResolverAdaptor::getReturnTypeDefaultImplementationF
} }
if (null_presence.has_nullable) if (null_presence.has_nullable)
{ {
Block nested_columns = createBlockWithNestedColumns(arguments); auto nested_columns = Block(createBlockWithNestedColumns(arguments));
auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end())); auto return_type = getter(ColumnsWithTypeAndName(nested_columns.begin(), nested_columns.end()));
return makeNullable(return_type); return makeNullable(return_type);
} }

View File

@ -17,7 +17,7 @@ private:
bool nextImpl() override; bool nextImpl() override;
public: public:
ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE); explicit ReadBufferFromIStream(std::istream & istr_, size_t size = DBMS_DEFAULT_BUFFER_SIZE);
}; };
} }

View File

@ -1,5 +1,6 @@
#pragma once #pragma once
#include <Core/UUID.h>
#include <Poco/Net/SocketAddress.h> #include <Poco/Net/SocketAddress.h>
#include <Common/UInt128.h> #include <Common/UInt128.h>
#include <common/types.h> #include <common/types.h>

View File

@ -2615,6 +2615,20 @@ PartUUIDsPtr Context::getPartUUIDs()
return part_uuids; return part_uuids;
} }
ReadTaskCallback Context::getReadTaskCallback() const
{
if (!next_task_callback.has_value())
throw Exception(fmt::format("Next task callback is not set for query {}", getInitialQueryId()), ErrorCodes::LOGICAL_ERROR);
return next_task_callback.value();
}
void Context::setReadTaskCallback(ReadTaskCallback && callback)
{
next_task_callback = callback;
}
PartUUIDsPtr Context::getIgnoredPartUUIDs() PartUUIDsPtr Context::getIgnoredPartUUIDs()
{ {
auto lock = getLock(); auto lock = getLock();

View File

@ -128,6 +128,9 @@ using InputInitializer = std::function<void(ContextPtr, const StoragePtr &)>;
/// Callback for reading blocks of data from client for function input() /// Callback for reading blocks of data from client for function input()
using InputBlocksReader = std::function<Block(ContextPtr)>; using InputBlocksReader = std::function<Block(ContextPtr)>;
/// Used in distributed task processing
using ReadTaskCallback = std::function<String()>;
/// An empty interface for an arbitrary object that may be attached by a shared pointer /// An empty interface for an arbitrary object that may be attached by a shared pointer
/// to query context, when using ClickHouse as a library. /// to query context, when using ClickHouse as a library.
struct IHostContext struct IHostContext
@ -189,6 +192,9 @@ private:
TemporaryTablesMapping external_tables_mapping; TemporaryTablesMapping external_tables_mapping;
Scalars scalars; Scalars scalars;
/// Fields for distributed s3 function
std::optional<ReadTaskCallback> next_task_callback;
/// Record entities accessed by current query, and store this information in system.query_log. /// Record entities accessed by current query, and store this information in system.query_log.
struct QueryAccessInfo struct QueryAccessInfo
{ {
@ -769,6 +775,10 @@ public:
PartUUIDsPtr getPartUUIDs(); PartUUIDsPtr getPartUUIDs();
PartUUIDsPtr getIgnoredPartUUIDs(); PartUUIDsPtr getIgnoredPartUUIDs();
ReadTaskCallback getReadTaskCallback() const;
void setReadTaskCallback(ReadTaskCallback && callback);
private: private:
std::unique_lock<std::recursive_mutex> getLock() const; std::unique_lock<std::recursive_mutex> getLock() const;

View File

@ -26,9 +26,9 @@ struct DatabaseAndTableWithAlias
UUID uuid = UUIDHelpers::Nil; UUID uuid = UUIDHelpers::Nil;
DatabaseAndTableWithAlias() = default; DatabaseAndTableWithAlias() = default;
DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = ""); explicit DatabaseAndTableWithAlias(const ASTPtr & identifier_node, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = ""); explicit DatabaseAndTableWithAlias(const ASTIdentifier & identifier, const String & current_database = "");
DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = ""); explicit DatabaseAndTableWithAlias(const ASTTableExpression & table_expression, const String & current_database = "");
/// "alias." or "table." if alias is empty /// "alias." or "table." if alias is empty
String getQualifiedNamePrefix(bool with_dot = true) const; String getQualifiedNamePrefix(bool with_dot = true) const;
@ -80,7 +80,7 @@ private:
void addAdditionalColumns(NamesAndTypesList & target, const NamesAndTypesList & addition) void addAdditionalColumns(NamesAndTypesList & target, const NamesAndTypesList & addition)
{ {
target.insert(target.end(), addition.begin(), addition.end()); target.insert(target.end(), addition.begin(), addition.end());
for (auto & col : addition) for (const auto & col : addition)
names.insert(col.name); names.insert(col.name);
} }

View File

@ -20,7 +20,7 @@ public:
bool second_with_brackets; bool second_with_brackets;
public: public:
ASTPair(bool second_with_brackets_) explicit ASTPair(bool second_with_brackets_)
: second_with_brackets(second_with_brackets_) : second_with_brackets(second_with_brackets_)
{ {
} }
@ -49,7 +49,7 @@ public:
/// Has brackets around arguments /// Has brackets around arguments
bool has_brackets; bool has_brackets;
ASTFunctionWithKeyValueArguments(bool has_brackets_ = true) explicit ASTFunctionWithKeyValueArguments(bool has_brackets_ = true)
: has_brackets(has_brackets_) : has_brackets(has_brackets_)
{ {
} }

View File

@ -45,7 +45,7 @@ protected:
class ParserIdentifier : public IParserBase class ParserIdentifier : public IParserBase
{ {
public: public:
ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {} explicit ParserIdentifier(bool allow_query_parameter_ = false) : allow_query_parameter(allow_query_parameter_) {}
protected: protected:
const char * getName() const override { return "identifier"; } const char * getName() const override { return "identifier"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
@ -59,7 +59,7 @@ protected:
class ParserCompoundIdentifier : public IParserBase class ParserCompoundIdentifier : public IParserBase
{ {
public: public:
ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false) explicit ParserCompoundIdentifier(bool table_name_with_optional_uuid_ = false, bool allow_query_parameter_ = false)
: table_name_with_optional_uuid(table_name_with_optional_uuid_), allow_query_parameter(allow_query_parameter_) : table_name_with_optional_uuid(table_name_with_optional_uuid_), allow_query_parameter(allow_query_parameter_)
{ {
} }
@ -85,7 +85,7 @@ public:
using ColumnTransformers = MultiEnum<ColumnTransformer, UInt8>; using ColumnTransformers = MultiEnum<ColumnTransformer, UInt8>;
static constexpr auto AllTransformers = ColumnTransformers{ColumnTransformer::APPLY, ColumnTransformer::EXCEPT, ColumnTransformer::REPLACE}; static constexpr auto AllTransformers = ColumnTransformers{ColumnTransformer::APPLY, ColumnTransformer::EXCEPT, ColumnTransformer::REPLACE};
ParserColumnsTransformers(ColumnTransformers allowed_transformers_ = AllTransformers, bool is_strict_ = false) explicit ParserColumnsTransformers(ColumnTransformers allowed_transformers_ = AllTransformers, bool is_strict_ = false)
: allowed_transformers(allowed_transformers_) : allowed_transformers(allowed_transformers_)
, is_strict(is_strict_) , is_strict(is_strict_)
{} {}
@ -103,7 +103,7 @@ class ParserAsterisk : public IParserBase
{ {
public: public:
using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers; using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers;
ParserAsterisk(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers) explicit ParserAsterisk(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
: allowed_transformers(allowed_transformers_) : allowed_transformers(allowed_transformers_)
{} {}
@ -129,7 +129,7 @@ class ParserColumnsMatcher : public IParserBase
{ {
public: public:
using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers; using ColumnTransformers = ParserColumnsTransformers::ColumnTransformers;
ParserColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers) explicit ParserColumnsMatcher(ColumnTransformers allowed_transformers_ = ParserColumnsTransformers::AllTransformers)
: allowed_transformers(allowed_transformers_) : allowed_transformers(allowed_transformers_)
{} {}
@ -149,7 +149,7 @@ protected:
class ParserFunction : public IParserBase class ParserFunction : public IParserBase
{ {
public: public:
ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false) explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_) : allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
{ {
} }

View File

@ -25,6 +25,7 @@
#include <Interpreters/OpenTelemetrySpanLog.h> #include <Interpreters/OpenTelemetrySpanLog.h>
#include <Storages/StorageReplicatedMergeTree.h> #include <Storages/StorageReplicatedMergeTree.h>
#include <Storages/MergeTree/MergeTreeDataPartUUID.h> #include <Storages/MergeTree/MergeTreeDataPartUUID.h>
#include <Storages/StorageS3Cluster.h>
#include <Core/ExternalTable.h> #include <Core/ExternalTable.h>
#include <Storages/ColumnDefault.h> #include <Storages/ColumnDefault.h>
#include <DataTypes/DataTypeLowCardinality.h> #include <DataTypes/DataTypeLowCardinality.h>
@ -33,6 +34,7 @@
#include <Processors/Executors/PullingAsyncPipelineExecutor.h> #include <Processors/Executors/PullingAsyncPipelineExecutor.h>
#include "Core/Protocol.h"
#include "TCPHandler.h" #include "TCPHandler.h"
#if !defined(ARCADIA_BUILD) #if !defined(ARCADIA_BUILD)
@ -55,6 +57,7 @@ namespace ErrorCodes
extern const int SOCKET_TIMEOUT; extern const int SOCKET_TIMEOUT;
extern const int UNEXPECTED_PACKET_FROM_CLIENT; extern const int UNEXPECTED_PACKET_FROM_CLIENT;
extern const int SUPPORT_IS_DISABLED; extern const int SUPPORT_IS_DISABLED;
extern const int UNKNOWN_PROTOCOL;
} }
TCPHandler::TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_) TCPHandler::TCPHandler(IServer & server_, const Poco::Net::StreamSocket & socket_, bool parse_proxy_protocol_, std::string server_display_name_)
@ -285,6 +288,14 @@ void TCPHandler::runImpl()
customizeContext(query_context); customizeContext(query_context);
/// This callback is needed for requesting read tasks inside pipeline for distributed processing
query_context->setReadTaskCallback([this]() -> String
{
std::lock_guard lock(task_callback_mutex);
sendReadTaskRequestAssumeLocked();
return receiveReadTaskResponseAssumeLocked();
});
bool may_have_embedded_data = client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA; bool may_have_embedded_data = client_tcp_protocol_version >= DBMS_MIN_REVISION_WITH_CLIENT_SUPPORT_EMBEDDED_DATA;
/// Processing Query /// Processing Query
state.io = executeQuery(state.query, query_context, false, state.stage, may_have_embedded_data); state.io = executeQuery(state.query, query_context, false, state.stage, may_have_embedded_data);
@ -644,6 +655,8 @@ void TCPHandler::processOrdinaryQueryWithProcessors()
Block block; Block block;
while (executor.pull(block, query_context->getSettingsRef().interactive_delay / 1000)) while (executor.pull(block, query_context->getSettingsRef().interactive_delay / 1000))
{ {
std::lock_guard lock(task_callback_mutex);
if (isQueryCancelled()) if (isQueryCancelled())
{ {
/// A packet was received requesting to stop execution of the request. /// A packet was received requesting to stop execution of the request.
@ -755,6 +768,13 @@ void TCPHandler::sendPartUUIDs()
} }
} }
void TCPHandler::sendReadTaskRequestAssumeLocked()
{
writeVarUInt(Protocol::Server::ReadTaskRequest, *out);
out->next();
}
void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info) void TCPHandler::sendProfileInfo(const BlockStreamProfileInfo & info)
{ {
writeVarUInt(Protocol::Server::ProfileInfo, *out); writeVarUInt(Protocol::Server::ProfileInfo, *out);
@ -963,10 +983,10 @@ bool TCPHandler::receivePacket()
UInt64 packet_type = 0; UInt64 packet_type = 0;
readVarUInt(packet_type, *in); readVarUInt(packet_type, *in);
// std::cerr << "Server got packet: " << Protocol::Client::toString(packet_type) << "\n";
switch (packet_type) switch (packet_type)
{ {
case Protocol::Client::ReadTaskResponse:
throw Exception("ReadTaskResponse must be received only after requesting in callback", ErrorCodes::LOGICAL_ERROR);
case Protocol::Client::IgnoredPartUUIDs: case Protocol::Client::IgnoredPartUUIDs:
/// Part uuids packet if any comes before query. /// Part uuids packet if any comes before query.
receiveIgnoredPartUUIDs(); receiveIgnoredPartUUIDs();
@ -1016,6 +1036,34 @@ void TCPHandler::receiveIgnoredPartUUIDs()
query_context->getIgnoredPartUUIDs()->add(uuids); query_context->getIgnoredPartUUIDs()->add(uuids);
} }
String TCPHandler::receiveReadTaskResponseAssumeLocked()
{
UInt64 packet_type = 0;
readVarUInt(packet_type, *in);
if (packet_type != Protocol::Client::ReadTaskResponse)
{
if (packet_type == Protocol::Client::Cancel)
{
state.is_cancelled = true;
return {};
}
else
{
throw Exception(fmt::format("Received {} packet after requesting read task",
Protocol::Client::toString(packet_type)), ErrorCodes::UNEXPECTED_PACKET_FROM_CLIENT);
}
}
UInt64 version;
readVarUInt(version, *in);
if (version != DBMS_CLUSTER_PROCESSING_PROTOCOL_VERSION)
throw Exception("Protocol version for distributed processing mismatched", ErrorCodes::UNKNOWN_PROTOCOL);
String response;
readStringBinary(response, *in);
return response;
}
void TCPHandler::receiveClusterNameAndSalt() void TCPHandler::receiveClusterNameAndSalt()
{ {
readStringBinary(cluster, *in); readStringBinary(cluster, *in);

View File

@ -89,7 +89,7 @@ struct QueryState
*this = QueryState(); *this = QueryState();
} }
bool empty() bool empty() const
{ {
return is_empty; return is_empty;
} }
@ -150,6 +150,7 @@ private:
String cluster; String cluster;
String cluster_secret; String cluster_secret;
std::mutex task_callback_mutex;
/// At the moment, only one ongoing query in the connection is supported at a time. /// At the moment, only one ongoing query in the connection is supported at a time.
QueryState state; QueryState state;
@ -169,9 +170,11 @@ private:
bool receivePacket(); bool receivePacket();
void receiveQuery(); void receiveQuery();
void receiveIgnoredPartUUIDs(); void receiveIgnoredPartUUIDs();
String receiveReadTaskResponseAssumeLocked();
bool receiveData(bool scalar); bool receiveData(bool scalar);
bool readDataNext(const size_t & poll_interval, const int & receive_timeout); bool readDataNext(const size_t & poll_interval, const int & receive_timeout);
void readData(const Settings & connection_settings); void readData(const Settings & connection_settings);
void receiveClusterNameAndSalt();
std::tuple<size_t, int> getReadTimeouts(const Settings & connection_settings); std::tuple<size_t, int> getReadTimeouts(const Settings & connection_settings);
[[noreturn]] void receiveUnexpectedData(); [[noreturn]] void receiveUnexpectedData();
@ -198,12 +201,11 @@ private:
void sendLogs(); void sendLogs();
void sendEndOfStream(); void sendEndOfStream();
void sendPartUUIDs(); void sendPartUUIDs();
void sendReadTaskRequestAssumeLocked();
void sendProfileInfo(const BlockStreamProfileInfo & info); void sendProfileInfo(const BlockStreamProfileInfo & info);
void sendTotals(const Block & totals); void sendTotals(const Block & totals);
void sendExtremes(const Block & extremes); void sendExtremes(const Block & extremes);
void receiveClusterNameAndSalt();
/// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled. /// Creates state.block_in/block_out for blocks read/write, depending on whether compression is enabled.
void initBlockInput(); void initBlockInput();
void initBlockOutput(const Block & block); void initBlockOutput(const Block & block);

View File

@ -452,7 +452,8 @@ StorageDistributed::StorageDistributed(
const DistributedSettings & distributed_settings_, const DistributedSettings & distributed_settings_,
bool attach, bool attach,
ClusterPtr owned_cluster_) ClusterPtr owned_cluster_)
: StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_, storage_policy_name_, relative_data_path_, distributed_settings_, attach, std::move(owned_cluster_)) : StorageDistributed(id_, columns_, constraints_, String{}, String{}, cluster_name_, context_, sharding_key_,
storage_policy_name_, relative_data_path_, distributed_settings_, attach, std::move(owned_cluster_))
{ {
remote_table_function_ptr = std::move(remote_table_function_ptr_); remote_table_function_ptr = std::move(remote_table_function_ptr_);
} }
@ -473,20 +474,15 @@ QueryProcessingStage::Enum StorageDistributed::getQueryProcessingStage(
ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query); ClusterPtr optimized_cluster = getOptimizedCluster(local_context, metadata_snapshot, query_info.query);
if (optimized_cluster) if (optimized_cluster)
{ {
LOG_DEBUG( LOG_DEBUG(log, "Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}",
log, makeFormattedListOfShards(optimized_cluster));
"Skipping irrelevant shards - the query will be sent to the following shards of the cluster (shard numbers): {}",
makeFormattedListOfShards(optimized_cluster));
cluster = optimized_cluster; cluster = optimized_cluster;
query_info.optimized_cluster = cluster; query_info.optimized_cluster = cluster;
} }
else else
{ {
LOG_DEBUG( LOG_DEBUG(log, "Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the cluster{}",
log, has_sharding_key ? "" : " (no sharding key)");
"Unable to figure out irrelevant shards from WHERE/PREWHERE clauses - the query will be sent to all shards of the "
"cluster{}",
has_sharding_key ? "" : " (no sharding key)");
} }
} }

View File

@ -11,7 +11,7 @@ class StorageProxy : public IStorage
{ {
public: public:
StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {} explicit StorageProxy(const StorageID & table_id_) : IStorage(table_id_) {}
virtual StoragePtr getNested() const = 0; virtual StoragePtr getNested() const = 0;

View File

@ -45,154 +45,267 @@ namespace ErrorCodes
extern const int UNEXPECTED_EXPRESSION; extern const int UNEXPECTED_EXPRESSION;
extern const int S3_ERROR; extern const int S3_ERROR;
} }
class StorageS3Source::DisclosedGlobIterator::Impl
namespace
{ {
class StorageS3Source : public SourceWithProgress
public:
Impl(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
: client(client_), globbed_uri(globbed_uri_)
{ {
public: std::lock_guard lock(mutex);
static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column) if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{"));
/// We don't have to list bucket, because there is no asterics.
if (key_prefix.size() == globbed_uri.key.size())
{ {
if (with_path_column) buffer.emplace_back(globbed_uri.key);
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"}); buffer_iter = buffer.begin();
if (with_file_column) is_finished = true;
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"}); return;
return sample_block;
} }
StorageS3Source( request.SetBucket(globbed_uri.bucket);
bool need_path, request.SetPrefix(key_prefix);
bool need_file, matcher = std::make_unique<re2::RE2>(makeRegexpPatternFromGlobs(globbed_uri.key));
const String & format, fillInternalBufferAssumeLocked();
String name_, }
const Block & sample_block,
ContextPtr context,
const ColumnsDescription & columns,
UInt64 max_block_size,
const CompressionMethod compression_method,
const std::shared_ptr<Aws::S3::S3Client> & client,
const String & bucket,
const String & key)
: SourceWithProgress(getHeader(sample_block, need_path, need_file))
, name(std::move(name_))
, with_file_column(need_file)
, with_path_column(need_path)
, file_path(bucket + "/" + key)
{
read_buf = wrapReadBufferWithCompressionMethod(std::make_unique<ReadBufferFromS3>(client, bucket, key), compression_method);
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
if (columns.hasDefaults()) String next()
reader = std::make_shared<AddingDefaultsBlockInputStream>(reader, columns, context); {
std::lock_guard lock(mutex);
return nextAssumeLocked();
}
private:
String nextAssumeLocked()
{
if (buffer_iter != buffer.end())
{
auto answer = *buffer_iter;
++buffer_iter;
return answer;
} }
String getName() const override if (is_finished)
{
return name;
}
Chunk generate() override
{
if (!reader)
return {};
if (!initialized)
{
reader->readSuffix();
initialized = true;
}
if (auto block = reader->read())
{
auto columns = block.getColumns();
UInt64 num_rows = block.rows();
if (with_path_column)
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
if (with_file_column)
{
size_t last_slash_pos = file_path.find_last_of('/');
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path.substr(
last_slash_pos + 1))->convertToFullColumnIfConst());
}
return Chunk(std::move(columns), num_rows);
}
reader.reset();
return {}; return {};
}
private: fillInternalBufferAssumeLocked();
String name;
std::unique_ptr<ReadBuffer> read_buf;
BlockInputStreamPtr reader;
bool initialized = false;
bool with_file_column = false;
bool with_path_column = false;
String file_path;
};
class StorageS3BlockOutputStream : public IBlockOutputStream return nextAssumeLocked();
}
void fillInternalBufferAssumeLocked()
{ {
public: buffer.clear();
StorageS3BlockOutputStream(
const String & format,
const Block & sample_block_,
ContextPtr context,
const CompressionMethod compression_method,
const std::shared_ptr<Aws::S3::S3Client> & client,
const String & bucket,
const String & key,
size_t min_upload_part_size,
size_t max_single_part_upload_size)
: sample_block(sample_block_)
{
write_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromS3>(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3);
writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context);
}
Block getHeader() const override outcome = client.ListObjectsV2(request);
{ if (!outcome.IsSuccess())
return sample_block; throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
} quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
void write(const Block & block) override const auto & result_batch = outcome.GetResult().GetContents();
{
writer->write(block);
}
void writePrefix() override buffer.reserve(result_batch.size());
for (const auto & row : result_batch)
{ {
writer->writePrefix(); String key = row.GetKey();
if (re2::RE2::FullMatch(key, *matcher))
buffer.emplace_back(std::move(key));
} }
/// Set iterator only after the whole batch is processed
buffer_iter = buffer.begin();
void flush() override request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
{
writer->flush();
}
void writeSuffix() override /// It returns false when all objects were returned
{ is_finished = !outcome.GetResult().GetIsTruncated();
writer->writeSuffix(); }
writer->flush();
write_buf->finalize();
}
private: std::mutex mutex;
Block sample_block; Strings buffer;
std::unique_ptr<WriteBuffer> write_buf; Strings::iterator buffer_iter;
BlockOutputStreamPtr writer; Aws::S3::S3Client client;
}; S3::URI globbed_uri;
Aws::S3::Model::ListObjectsV2Request request;
Aws::S3::Model::ListObjectsV2Outcome outcome;
std::unique_ptr<re2::RE2> matcher;
bool is_finished{false};
};
StorageS3Source::DisclosedGlobIterator::DisclosedGlobIterator(Aws::S3::S3Client & client_, const S3::URI & globbed_uri_)
: pimpl(std::make_shared<StorageS3Source::DisclosedGlobIterator::Impl>(client_, globbed_uri_)) {}
String StorageS3Source::DisclosedGlobIterator::next()
{
return pimpl->next();
} }
Block StorageS3Source::getHeader(Block sample_block, bool with_path_column, bool with_file_column)
{
if (with_path_column)
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_path"});
if (with_file_column)
sample_block.insert({DataTypeString().createColumn(), std::make_shared<DataTypeString>(), "_file"});
return sample_block;
}
StorageS3Source::StorageS3Source(
bool need_path,
bool need_file,
const String & format_,
String name_,
const Block & sample_block_,
ContextPtr context_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
const String compression_hint_,
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket_,
std::shared_ptr<IteratorWrapper> file_iterator_)
: SourceWithProgress(getHeader(sample_block_, need_path, need_file))
, WithContext(context_)
, name(std::move(name_))
, bucket(bucket_)
, format(format_)
, columns_desc(columns_)
, max_block_size(max_block_size_)
, compression_hint(compression_hint_)
, client(client_)
, sample_block(sample_block_)
, with_file_column(need_file)
, with_path_column(need_path)
, file_iterator(file_iterator_)
{
initialize();
}
bool StorageS3Source::initialize()
{
String current_key = (*file_iterator)();
if (current_key.empty())
return false;
file_path = bucket + "/" + current_key;
read_buf = wrapReadBufferWithCompressionMethod(
std::make_unique<ReadBufferFromS3>(client, bucket, current_key), chooseCompressionMethod(current_key, compression_hint));
auto input_format = FormatFactory::instance().getInput(format, *read_buf, sample_block, getContext(), max_block_size);
reader = std::make_shared<InputStreamFromInputFormat>(input_format);
if (columns_desc.hasDefaults())
reader = std::make_shared<AddingDefaultsBlockInputStream>(reader, columns_desc, getContext());
initialized = false;
return true;
}
String StorageS3Source::getName() const
{
return name;
}
Chunk StorageS3Source::generate()
{
if (!reader)
return {};
if (!initialized)
{
reader->readPrefix();
initialized = true;
}
if (auto block = reader->read())
{
auto columns = block.getColumns();
UInt64 num_rows = block.rows();
if (with_path_column)
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path)->convertToFullColumnIfConst());
if (with_file_column)
{
size_t last_slash_pos = file_path.find_last_of('/');
columns.push_back(DataTypeString().createColumnConst(num_rows, file_path.substr(
last_slash_pos + 1))->convertToFullColumnIfConst());
}
return Chunk(std::move(columns), num_rows);
}
reader->readSuffix();
reader.reset();
read_buf.reset();
if (!initialize())
return {};
return generate();
}
class StorageS3BlockOutputStream : public IBlockOutputStream
{
public:
StorageS3BlockOutputStream(
const String & format,
const Block & sample_block_,
ContextPtr context,
const CompressionMethod compression_method,
const std::shared_ptr<Aws::S3::S3Client> & client,
const String & bucket,
const String & key,
size_t min_upload_part_size,
size_t max_single_part_upload_size)
: sample_block(sample_block_)
{
write_buf = wrapWriteBufferWithCompressionMethod(
std::make_unique<WriteBufferFromS3>(client, bucket, key, min_upload_part_size, max_single_part_upload_size), compression_method, 3);
writer = FormatFactory::instance().getOutputStreamParallelIfPossible(format, *write_buf, sample_block, context);
}
Block getHeader() const override
{
return sample_block;
}
void write(const Block & block) override
{
writer->write(block);
}
void writePrefix() override
{
writer->writePrefix();
}
void flush() override
{
writer->flush();
}
void writeSuffix() override
{
writer->writeSuffix();
writer->flush();
write_buf->finalize();
}
private:
Block sample_block;
std::unique_ptr<WriteBuffer> write_buf;
BlockOutputStreamPtr writer;
};
StorageS3::StorageS3( StorageS3::StorageS3(
const S3::URI & uri_, const S3::URI & uri_,
const String & access_key_id_, const String & access_key_id_,
@ -205,84 +318,23 @@ StorageS3::StorageS3(
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
ContextPtr context_, ContextPtr context_,
const String & compression_method_) const String & compression_method_,
bool distributed_processing_)
: IStorage(table_id_) : IStorage(table_id_)
, WithContext(context_->getGlobalContext()) , client_auth{uri_, access_key_id_, secret_access_key_, max_connections_, {}, {}} /// Client and settings will be updated later
, uri(uri_)
, access_key_id(access_key_id_)
, secret_access_key(secret_access_key_)
, max_connections(max_connections_)
, format_name(format_name_) , format_name(format_name_)
, min_upload_part_size(min_upload_part_size_) , min_upload_part_size(min_upload_part_size_)
, max_single_part_upload_size(max_single_part_upload_size_) , max_single_part_upload_size(max_single_part_upload_size_)
, compression_method(compression_method_) , compression_method(compression_method_)
, name(uri_.storage_name) , name(uri_.storage_name)
, distributed_processing(distributed_processing_)
{ {
getContext()->getRemoteHostFilter().checkURL(uri_.uri); context_->getGlobalContext()->getRemoteHostFilter().checkURL(uri_.uri);
StorageInMemoryMetadata storage_metadata; StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_); storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_); storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata); setInMemoryMetadata(storage_metadata);
updateAuthSettings(context_); updateClientAndAuthSettings(context_, client_auth);
}
namespace
{
/* "Recursive" directory listing with matched paths as a result.
* Have the same method in StorageFile.
*/
Strings listFilesWithRegexpMatching(Aws::S3::S3Client & client, const S3::URI & globbed_uri)
{
if (globbed_uri.bucket.find_first_of("*?{") != globbed_uri.bucket.npos)
{
throw Exception("Expression can not have wildcards inside bucket name", ErrorCodes::UNEXPECTED_EXPRESSION);
}
const String key_prefix = globbed_uri.key.substr(0, globbed_uri.key.find_first_of("*?{"));
if (key_prefix.size() == globbed_uri.key.size())
{
return {globbed_uri.key};
}
Aws::S3::Model::ListObjectsV2Request request;
request.SetBucket(globbed_uri.bucket);
request.SetPrefix(key_prefix);
re2::RE2 matcher(makeRegexpPatternFromGlobs(globbed_uri.key));
Strings result;
Aws::S3::Model::ListObjectsV2Outcome outcome;
int page = 0;
do
{
++page;
outcome = client.ListObjectsV2(request);
if (!outcome.IsSuccess())
{
if (page > 1)
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, page {}, S3 exception: {}, message: {}",
quoteString(request.GetBucket()), quoteString(request.GetPrefix()), page,
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
throw Exception(ErrorCodes::S3_ERROR, "Could not list objects in bucket {} with prefix {}, S3 exception: {}, message: {}",
quoteString(request.GetBucket()), quoteString(request.GetPrefix()),
backQuote(outcome.GetError().GetExceptionName()), quoteString(outcome.GetError().GetMessage()));
}
for (const auto & row : outcome.GetResult().GetContents())
{
String key = row.GetKey();
if (re2::RE2::FullMatch(key, matcher))
result.emplace_back(std::move(key));
}
request.SetContinuationToken(outcome.GetResult().GetNextContinuationToken());
}
while (outcome.GetResult().GetIsTruncated());
return result;
}
} }
@ -295,7 +347,7 @@ Pipe StorageS3::read(
size_t max_block_size, size_t max_block_size,
unsigned num_streams) unsigned num_streams)
{ {
updateAuthSettings(local_context); updateClientAndAuthSettings(local_context, client_auth);
Pipes pipes; Pipes pipes;
bool need_path_column = false; bool need_path_column = false;
@ -308,7 +360,26 @@ Pipe StorageS3::read(
need_file_column = true; need_file_column = true;
} }
for (const String & key : listFilesWithRegexpMatching(*client, uri)) std::shared_ptr<StorageS3Source::IteratorWrapper> iterator_wrapper{nullptr};
if (distributed_processing)
{
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>(
[callback = local_context->getReadTaskCallback()]() -> String {
return callback();
});
}
else
{
/// Iterate through disclosed globs and make a source for each file
auto glob_iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
iterator_wrapper = std::make_shared<StorageS3Source::IteratorWrapper>([glob_iterator]()
{
return glob_iterator->next();
});
}
for (size_t i = 0; i < num_streams; ++i)
{
pipes.emplace_back(std::make_shared<StorageS3Source>( pipes.emplace_back(std::make_shared<StorageS3Source>(
need_path_column, need_path_column,
need_file_column, need_file_column,
@ -318,63 +389,62 @@ Pipe StorageS3::read(
local_context, local_context,
metadata_snapshot->getColumns(), metadata_snapshot->getColumns(),
max_block_size, max_block_size,
chooseCompressionMethod(uri.key, compression_method), compression_method,
client, client_auth.client,
uri.bucket, client_auth.uri.bucket,
key)); iterator_wrapper));
}
auto pipe = Pipe::unitePipes(std::move(pipes)); auto pipe = Pipe::unitePipes(std::move(pipes));
// It's possible to have many buckets read from s3, resize(num_streams) might open too many handles at the same time.
// Using narrowPipe instead.
narrowPipe(pipe, num_streams); narrowPipe(pipe, num_streams);
return pipe; return pipe;
} }
BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context) BlockOutputStreamPtr StorageS3::write(const ASTPtr & /*query*/, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
{ {
updateAuthSettings(local_context); updateClientAndAuthSettings(local_context, client_auth);
return std::make_shared<StorageS3BlockOutputStream>( return std::make_shared<StorageS3BlockOutputStream>(
format_name, format_name,
metadata_snapshot->getSampleBlock(), metadata_snapshot->getSampleBlock(),
getContext(), local_context,
chooseCompressionMethod(uri.key, compression_method), chooseCompressionMethod(client_auth.uri.key, compression_method),
client, client_auth.client,
uri.bucket, client_auth.uri.bucket,
uri.key, client_auth.uri.key,
min_upload_part_size, min_upload_part_size,
max_single_part_upload_size); max_single_part_upload_size);
} }
void StorageS3::updateAuthSettings(ContextPtr local_context) void StorageS3::updateClientAndAuthSettings(ContextPtr ctx, StorageS3::ClientAuthentificaiton & upd)
{ {
auto settings = local_context->getStorageS3Settings().getSettings(uri.uri.toString()); auto settings = ctx->getStorageS3Settings().getSettings(upd.uri.uri.toString());
if (client && (!access_key_id.empty() || settings == auth_settings)) if (upd.client && (!upd.access_key_id.empty() || settings == upd.auth_settings))
return; return;
Aws::Auth::AWSCredentials credentials(access_key_id, secret_access_key); Aws::Auth::AWSCredentials credentials(upd.access_key_id, upd.secret_access_key);
HeaderCollection headers; HeaderCollection headers;
if (access_key_id.empty()) if (upd.access_key_id.empty())
{ {
credentials = Aws::Auth::AWSCredentials(settings.access_key_id, settings.secret_access_key); credentials = Aws::Auth::AWSCredentials(settings.access_key_id, settings.secret_access_key);
headers = settings.headers; headers = settings.headers;
} }
S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration( S3::PocoHTTPClientConfiguration client_configuration = S3::ClientFactory::instance().createClientConfiguration(
local_context->getRemoteHostFilter(), local_context->getGlobalContext()->getSettingsRef().s3_max_redirects); ctx->getRemoteHostFilter(), ctx->getGlobalContext()->getSettingsRef().s3_max_redirects);
client_configuration.endpointOverride = uri.endpoint; client_configuration.endpointOverride = upd.uri.endpoint;
client_configuration.maxConnections = max_connections; client_configuration.maxConnections = upd.max_connections;
client = S3::ClientFactory::instance().create( upd.client = S3::ClientFactory::instance().create(
client_configuration, client_configuration,
uri.is_virtual_hosted_style, upd.uri.is_virtual_hosted_style,
credentials.GetAWSAccessKeyId(), credentials.GetAWSAccessKeyId(),
credentials.GetAWSSecretKey(), credentials.GetAWSSecretKey(),
settings.server_side_encryption_customer_key_base64, settings.server_side_encryption_customer_key_base64,
std::move(headers), std::move(headers),
settings.use_environment_credentials.value_or(getContext()->getConfigRef().getBool("s3.use_environment_credentials", false))); settings.use_environment_credentials.value_or(ctx->getConfigRef().getBool("s3.use_environment_credentials", false)));
auth_settings = std::move(settings); upd.auth_settings = std::move(settings);
} }
void registerStorageS3Impl(const String & name, StorageFactory & factory) void registerStorageS3Impl(const String & name, StorageFactory & factory)
@ -385,7 +455,8 @@ void registerStorageS3Impl(const String & name, StorageFactory & factory)
if (engine_args.size() < 2 || engine_args.size() > 5) if (engine_args.size() < 2 || engine_args.size() > 5)
throw Exception( throw Exception(
"Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH); "Storage S3 requires 2 to 5 arguments: url, [access_key_id, secret_access_key], name of used format and [compression_method].",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & engine_arg : engine_args) for (auto & engine_arg : engine_args)
engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext()); engine_arg = evaluateConstantExpressionOrIdentifierAsLiteral(engine_arg, args.getLocalContext());

View File

@ -4,11 +4,20 @@
#if USE_AWS_S3 #if USE_AWS_S3
#include <Core/Types.h>
#include <Compression/CompressionInfo.h>
#include <Storages/IStorage.h> #include <Storages/IStorage.h>
#include <Storages/StorageS3Settings.h> #include <Storages/StorageS3Settings.h>
#include <Processors/Sources/SourceWithProgress.h>
#include <Poco/URI.h> #include <Poco/URI.h>
#include <common/logger_useful.h> #include <common/logger_useful.h>
#include <ext/shared_ptr_helper.h> #include <ext/shared_ptr_helper.h>
#include <IO/S3Common.h>
#include <IO/CompressionMethod.h>
#include <Interpreters/Context.h>
namespace Aws::S3 namespace Aws::S3
{ {
@ -18,6 +27,66 @@ namespace Aws::S3
namespace DB namespace DB
{ {
class StorageS3SequentialSource;
class StorageS3Source : public SourceWithProgress, WithContext
{
public:
class DisclosedGlobIterator
{
public:
DisclosedGlobIterator(Aws::S3::S3Client &, const S3::URI &);
String next();
private:
class Impl;
/// shared_ptr to have copy constructor
std::shared_ptr<Impl> pimpl;
};
using IteratorWrapper = std::function<String()>;
static Block getHeader(Block sample_block, bool with_path_column, bool with_file_column);
StorageS3Source(
bool need_path,
bool need_file,
const String & format,
String name_,
const Block & sample_block,
ContextPtr context_,
const ColumnsDescription & columns_,
UInt64 max_block_size_,
const String compression_hint_,
const std::shared_ptr<Aws::S3::S3Client> & client_,
const String & bucket,
std::shared_ptr<IteratorWrapper> file_iterator_);
String getName() const override;
Chunk generate() override;
private:
String name;
String bucket;
String file_path;
String format;
ColumnsDescription columns_desc;
UInt64 max_block_size;
String compression_hint;
std::shared_ptr<Aws::S3::S3Client> client;
Block sample_block;
std::unique_ptr<ReadBuffer> read_buf;
BlockInputStreamPtr reader;
bool initialized = false;
bool with_file_column = false;
bool with_path_column = false;
std::shared_ptr<IteratorWrapper> file_iterator;
/// Recreate ReadBuffer and BlockInputStream for each file.
bool initialize();
};
/** /**
* This class represents table engine for external S3 urls. * This class represents table engine for external S3 urls.
* It sends HTTP GET to server when select is called and * It sends HTTP GET to server when select is called and
@ -37,7 +106,8 @@ public:
const ColumnsDescription & columns_, const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_, const ConstraintsDescription & constraints_,
ContextPtr context_, ContextPtr context_,
const String & compression_method_ = ""); const String & compression_method_ = "",
bool distributed_processing_ = false);
String getName() const override String getName() const override
{ {
@ -58,20 +128,30 @@ public:
NamesAndTypesList getVirtuals() const override; NamesAndTypesList getVirtuals() const override;
private: private:
const S3::URI uri;
const String access_key_id; friend class StorageS3Cluster;
const String secret_access_key; friend class TableFunctionS3Cluster;
const UInt64 max_connections;
struct ClientAuthentificaiton
{
const S3::URI uri;
const String access_key_id;
const String secret_access_key;
const UInt64 max_connections;
std::shared_ptr<Aws::S3::S3Client> client;
S3AuthSettings auth_settings;
};
ClientAuthentificaiton client_auth;
String format_name; String format_name;
size_t min_upload_part_size; size_t min_upload_part_size;
size_t max_single_part_upload_size; size_t max_single_part_upload_size;
String compression_method; String compression_method;
std::shared_ptr<Aws::S3::S3Client> client;
String name; String name;
S3AuthSettings auth_settings; const bool distributed_processing;
void updateAuthSettings(ContextPtr context); static void updateClientAndAuthSettings(ContextPtr, ClientAuthentificaiton &);
}; };
} }

View File

@ -0,0 +1,166 @@
#include "Storages/StorageS3Cluster.h"
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AWS_S3
#include "Common/Exception.h"
#include <Common/Throttler.h>
#include "Client/Connection.h"
#include "Core/QueryProcessingStage.h"
#include <Core/UUID.h>
#include "DataStreams/RemoteBlockInputStream.h"
#include <Columns/ColumnsNumber.h>
#include <DataTypes/DataTypesNumber.h>
#include <DataTypes/DataTypeString.h>
#include <IO/ReadBufferFromS3.h>
#include <IO/ReadHelpers.h>
#include <IO/WriteBufferFromS3.h>
#include <IO/WriteHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/getHeaderForProcessingStage.h>
#include <Interpreters/SelectQueryOptions.h>
#include <Interpreters/InterpreterSelectQuery.h>
#include <Interpreters/getTableExpressions.h>
#include <Formats/FormatFactory.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/AddingDefaultsBlockInputStream.h>
#include <DataStreams/narrowBlockInputStreams.h>
#include <Processors/Formats/InputStreamFromInputFormat.h>
#include <Processors/Pipe.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include "Processors/Sources/SourceWithProgress.h"
#include <Processors/Sources/RemoteSource.h>
#include <Parsers/queryToString.h>
#include <Parsers/ASTTablesInSelectQuery.h>
#include <Storages/IStorage.h>
#include <Storages/SelectQueryInfo.h>
#include <common/logger_useful.h>
#include <aws/core/auth/AWSCredentials.h>
#include <aws/s3/S3Client.h>
#include <aws/s3/model/ListObjectsV2Request.h>
#include <ios>
#include <memory>
#include <string>
#include <thread>
#include <cassert>
namespace DB
{
StorageS3Cluster::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_)
: IStorage(table_id_)
, client_auth{S3::URI{Poco::URI{filename_}}, access_key_id_, secret_access_key_, max_connections_, {}, {}}
, filename(filename_)
, cluster_name(cluster_name_)
, format_name(format_name_)
, compression_method(compression_method_)
{
StorageInMemoryMetadata storage_metadata;
storage_metadata.setColumns(columns_);
storage_metadata.setConstraints(constraints_);
setInMemoryMetadata(storage_metadata);
StorageS3::updateClientAndAuthSettings(context_, client_auth);
}
/// The code executes on initiator
Pipe StorageS3Cluster::read(
const Names & column_names,
const StorageMetadataPtr & metadata_snapshot,
SelectQueryInfo & query_info,
ContextPtr context,
QueryProcessingStage::Enum processed_stage,
size_t /*max_block_size*/,
unsigned /*num_streams*/)
{
StorageS3::updateClientAndAuthSettings(context, client_auth);
auto cluster = context->getCluster(cluster_name)->getClusterWithReplicasAsShards(context->getSettings());
S3::URI s3_uri(Poco::URI{filename});
StorageS3::updateClientAndAuthSettings(context, client_auth);
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(*client_auth.client, client_auth.uri);
auto callback = std::make_shared<StorageS3Source::IteratorWrapper>([iterator]() mutable -> String
{
return iterator->next();
});
/// Calculate the header. This is significant, because some columns could be thrown away in some cases like query with count(*)
Block header =
InterpreterSelectQuery(query_info.query, context, SelectQueryOptions(processed_stage).analyze()).getSampleBlock();
const Scalars & scalars = context->hasQueryContext() ? context->getQueryContext()->getScalars() : Scalars{};
Pipes pipes;
connections.reserve(cluster->getShardCount());
const bool add_agg_info = processed_stage == QueryProcessingStage::WithMergeableState;
for (const auto & replicas : cluster->getShardsAddresses())
{
/// There will be only one replica, because we consider each replica as a shard
for (const auto & node : replicas)
{
connections.emplace_back(std::make_shared<Connection>(
node.host_name, node.port, context->getGlobalContext()->getCurrentDatabase(),
node.user, node.password, node.cluster, node.cluster_secret,
"S3ClusterInititiator",
node.compression,
node.secure
));
/// For unknown reason global context is passed to IStorage::read() method
/// So, task_identifier is passed as constructor argument. It is more obvious.
auto remote_query_executor = std::make_shared<RemoteQueryExecutor>(
*connections.back(), queryToString(query_info.query), header, context,
/*throttler=*/nullptr, scalars, Tables(), processed_stage, callback);
pipes.emplace_back(std::make_shared<RemoteSource>(remote_query_executor, add_agg_info, false));
}
}
metadata_snapshot->check(column_names, getVirtuals(), getStorageID());
return Pipe::unitePipes(std::move(pipes));
}
QueryProcessingStage::Enum StorageS3Cluster::getQueryProcessingStage(
ContextPtr context, QueryProcessingStage::Enum to_stage, SelectQueryInfo &) const
{
/// Initiator executes query on remote node.
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::INITIAL_QUERY)
if (to_stage >= QueryProcessingStage::Enum::WithMergeableState)
return QueryProcessingStage::Enum::WithMergeableState;
/// Follower just reads the data.
return QueryProcessingStage::Enum::FetchColumns;
}
NamesAndTypesList StorageS3Cluster::getVirtuals() const
{
return NamesAndTypesList{
{"_path", std::make_shared<DataTypeString>()},
{"_file", std::make_shared<DataTypeString>()}
};
}
}
#endif

View File

@ -0,0 +1,63 @@
#pragma once
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AWS_S3
#include "Client/Connection.h"
#include <Interpreters/Cluster.h>
#include <IO/S3Common.h>
#include <Storages/StorageS3.h>
#include <memory>
#include <optional>
#include "ext/shared_ptr_helper.h"
namespace DB
{
class Context;
struct ClientAuthentificationBuilder
{
String access_key_id;
String secret_access_key;
UInt64 max_connections;
};
class StorageS3Cluster : public ext::shared_ptr_helper<StorageS3Cluster>, public IStorage
{
friend struct ext::shared_ptr_helper<StorageS3Cluster>;
public:
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;
QueryProcessingStage::Enum getQueryProcessingStage(ContextPtr, QueryProcessingStage::Enum, SelectQueryInfo &) const override;
NamesAndTypesList getVirtuals() const override;
protected:
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_);
private:
/// Connections from initiator to other nodes
std::vector<std::shared_ptr<Connection>> connections;
StorageS3::ClientAuthentificaiton client_auth;
String filename;
String cluster_name;
String format_name;
String compression_method;
};
}
#endif

View File

@ -31,7 +31,7 @@ public:
unsigned num_streams) override; unsigned num_streams) override;
protected: protected:
StorageSystemOne(const StorageID & table_id_); explicit StorageSystemOne(const StorageID & table_id_);
}; };
} }

View File

@ -18,7 +18,7 @@ namespace DB
class TableFunctionRemote : public ITableFunction class TableFunctionRemote : public ITableFunction
{ {
public: public:
TableFunctionRemote(const std::string & name_, bool secure_ = false); explicit TableFunctionRemote(const std::string & name_, bool secure_ = false);
std::string getName() const override { return name; } std::string getName() const override { return name; }

View File

@ -17,7 +17,6 @@ namespace DB
namespace ErrorCodes namespace ErrorCodes
{ {
extern const int LOGICAL_ERROR;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH; extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
} }
@ -26,35 +25,53 @@ void TableFunctionS3::parseArguments(const ASTPtr & ast_function, ContextPtr con
/// Parse args /// Parse args
ASTs & args_func = ast_function->children; ASTs & args_func = ast_function->children;
const auto message = fmt::format(
"The signature of table function {} could be the following:\n" \
" - url, format, structure\n" \
" - url, format, structure, compression_method\n" \
" - url, access_key_id, secret_access_key, format, structure\n" \
" - url, access_key_id, secret_access_key, format, structure, compression_method",
getName());
if (args_func.size() != 1) if (args_func.size() != 1)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::LOGICAL_ERROR); throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTs & args = args_func.at(0)->children; ASTs & args = args_func.at(0)->children;
if (args.size() < 3 || args.size() > 6) if (args.size() < 3 || args.size() > 6)
throw Exception("Table function '" + getName() + "' requires 3 to 6 arguments: url, [access_key_id, secret_access_key,] format, structure and [compression_method].", throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args) for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context); arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// Size -> argument indexes
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
{
{3, {{"format", 1}, {"structure", 2}}},
{4, {{"format", 1}, {"structure", 2}, {"compression_method", 3}}},
{5, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}}},
{6, {{"access_key_id", 1}, {"secret_access_key", 2}, {"format", 3}, {"structure", 4}, {"compression_method", 5}}}
};
/// This argument is always the first
filename = args[0]->as<ASTLiteral &>().value.safeGet<String>(); filename = args[0]->as<ASTLiteral &>().value.safeGet<String>();
if (args.size() < 5) auto & args_to_idx = size_to_args[args.size()];
{
format = args[1]->as<ASTLiteral &>().value.safeGet<String>();
structure = args[2]->as<ASTLiteral &>().value.safeGet<String>();
}
else
{
access_key_id = args[1]->as<ASTLiteral &>().value.safeGet<String>();
secret_access_key = args[2]->as<ASTLiteral &>().value.safeGet<String>();
format = args[3]->as<ASTLiteral &>().value.safeGet<String>();
structure = args[4]->as<ASTLiteral &>().value.safeGet<String>();
}
if (args.size() == 4 || args.size() == 6) if (args_to_idx.contains("format"))
compression_method = args.back()->as<ASTLiteral &>().value.safeGet<String>(); format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("structure"))
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("compression_method"))
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("access_key_id"))
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("secret_access_key"))
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
} }
ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const ColumnsDescription TableFunctionS3::getActualTableStructure(ContextPtr context) const

View File

@ -0,0 +1,144 @@
#if !defined(ARCADIA_BUILD)
#include <Common/config.h>
#endif
#if USE_AWS_S3
#include <Storages/StorageS3Cluster.h>
#include <DataTypes/DataTypeString.h>
#include <DataStreams/RemoteBlockInputStream.h>
#include <IO/S3Common.h>
#include <Storages/StorageS3.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/Context.h>
#include <Interpreters/ClientInfo.h>
#include <TableFunctions/TableFunctionFactory.h>
#include <TableFunctions/TableFunctionS3.h>
#include <TableFunctions/TableFunctionS3Cluster.h>
#include <TableFunctions/parseColumnsListForTableFunction.h>
#include <Parsers/ASTLiteral.h>
#include <Parsers/ASTExpressionList.h>
#include <Parsers/ASTFunction.h>
#include <Parsers/IAST_fwd.h>
#include <Processors/Sources/SourceFromInputStream.h>
#include "registerTableFunctions.h"
#include <memory>
#include <thread>
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
void TableFunctionS3Cluster::parseArguments(const ASTPtr & ast_function, ContextPtr context)
{
/// Parse args
ASTs & args_func = ast_function->children;
if (args_func.size() != 1)
throw Exception("Table function '" + getName() + "' must have arguments.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
ASTs & args = args_func.at(0)->children;
const auto message = fmt::format(
"The signature of table function {} could be the following:\n" \
" - cluster, url, format, structure\n" \
" - cluster, url, format, structure, compression_method\n" \
" - cluster, url, access_key_id, secret_access_key, format, structure\n" \
" - cluster, url, access_key_id, secret_access_key, format, structure, compression_method",
getName());
if (args.size() < 4 || args.size() > 7)
throw Exception(message, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
for (auto & arg : args)
arg = evaluateConstantExpressionOrIdentifierAsLiteral(arg, context);
/// This arguments are always the first
cluster_name = args[0]->as<ASTLiteral &>().value.safeGet<String>();
filename = args[1]->as<ASTLiteral &>().value.safeGet<String>();
/// Size -> argument indexes
static auto size_to_args = std::map<size_t, std::map<String, size_t>>
{
{4, {{"format", 2}, {"structure", 3}}},
{5, {{"format", 2}, {"structure", 3}, {"compression_method", 4}}},
{6, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}}},
{7, {{"access_key_id", 2}, {"secret_access_key", 3}, {"format", 4}, {"structure", 5}, {"compression_method", 6}}}
};
auto & args_to_idx = size_to_args[args.size()];
if (args_to_idx.contains("format"))
format = args[args_to_idx["format"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("structure"))
structure = args[args_to_idx["structure"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("compression_method"))
compression_method = args[args_to_idx["compression_method"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("access_key_id"))
access_key_id = args[args_to_idx["access_key_id"]]->as<ASTLiteral &>().value.safeGet<String>();
if (args_to_idx.contains("secret_access_key"))
secret_access_key = args[args_to_idx["secret_access_key"]]->as<ASTLiteral &>().value.safeGet<String>();
}
ColumnsDescription TableFunctionS3Cluster::getActualTableStructure(ContextPtr context) const
{
return parseColumnsListFromString(structure, context);
}
StoragePtr TableFunctionS3Cluster::executeImpl(
const ASTPtr & /*function*/, ContextPtr context,
const std::string & table_name, ColumnsDescription /*cached_columns*/) const
{
StoragePtr storage;
if (context->getClientInfo().query_kind == ClientInfo::QueryKind::SECONDARY_QUERY)
{
/// On worker node this filename won't contains globs
Poco::URI uri (filename);
S3::URI s3_uri (uri);
/// Actually this parameters are not used
UInt64 min_upload_part_size = context->getSettingsRef().s3_min_upload_part_size;
UInt64 max_single_part_upload_size = context->getSettingsRef().s3_max_single_part_upload_size;
UInt64 max_connections = context->getSettingsRef().s3_max_connections;
storage = StorageS3::create(
s3_uri, access_key_id, secret_access_key, StorageID(getDatabaseName(), table_name),
format, min_upload_part_size, max_single_part_upload_size, max_connections,
getActualTableStructure(context), ConstraintsDescription{},
context, compression_method, /*distributed_processing=*/true);
}
else
{
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{},
context, compression_method);
}
storage->startup();
return storage;
}
void registerTableFunctionS3Cluster(TableFunctionFactory & factory)
{
factory.registerFunction<TableFunctionS3Cluster>();
}
}
#endif

View File

@ -0,0 +1,56 @@
#pragma once
#include <Common/config.h>
#if USE_AWS_S3
#include <TableFunctions/ITableFunction.h>
namespace DB
{
class Context;
/**
* 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 dispatch each file dynamically.
* On worker node it asks initiator about next task to process, processes it.
* This is repeated until the tasks are finished.
*/
class TableFunctionS3Cluster : public ITableFunction
{
public:
static constexpr auto name = "s3Cluster";
std::string getName() const override
{
return name;
}
bool hasStaticStructure() const override { return true; }
protected:
StoragePtr executeImpl(
const ASTPtr & ast_function,
ContextPtr context,
const std::string & table_name,
ColumnsDescription cached_columns) const override;
const char * getStorageTypeName() const override { return "S3Cluster"; }
ColumnsDescription getActualTableStructure(ContextPtr) const override;
void parseArguments(const ASTPtr &, ContextPtr) override;
String cluster_name;
String filename;
String format;
String structure;
String access_key_id;
String secret_access_key;
String compression_method = "auto";
};
}
#endif

View File

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

View File

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

View File

@ -0,0 +1,24 @@
<yandex>
<remote_servers>
<cluster_simple>
<shard>
<replica>
<host>s0_0_0</host>
<port>9000</port>
</replica>
<replica>
<host>s0_0_1</host>
<port>9000</port>
</replica>
</shard>
<shard>
<replica>
<host>s0_1_0</host>
<port>9000</port>
</replica>
</shard>
</cluster_simple>
</remote_servers>
</yandex>

View File

@ -0,0 +1,10 @@
"fSRH",976027584,"[[(-1.5346513608456012e-204,-2.867937504545497e266),(3.1627675144114637e-231,-2.20343471241604e-54),(-1.866886218651809e-89,-7.695893036366416e100),(8.196307577166986e-169,-8.203793887684096e-263),(-1.6150328830402252e-215,8.531116551449711e-296),(4.3378407855931477e92,1.1313645428723989e117),(-4.238081208165573e137,-8.969951719788361e67)],[(-3.409639554701108e169,-7.277093176871153e-254),(1.1466207153308928e-226,3.429893348531029e96),(6.451302850199177e-189,-7.52379443153242e125),(-1.7132078539493614e-127,-2.3177814806867505e241),(1.4996520594989919e-257,4.271017883966942e128)],[(65460976657479156000,1.7055814145588595e253),(-1.921491101580189e154,3.2912740465446566e-286),(0.0008437955075350972,-5.143493717005472e-107),(8.637208599142187e-150,7.825076274945548e136),(1.8077733932468565e-159,5.51061479974026e-77),(1.300406236793709e-260,10669142.497111017),(-1.731981751951159e91,-1.270795062098902e102)],[(3.336706342781395e-7,-1.1919528866481513e266)]]"
"sX6>",733011552,"[[(-3.737863336077909e-44,3.066510481088993e-161),(-1.0047259170558555e-31,8.066145272086467e-274)],[(1.2261835328136691e-58,-6.154561379350395e258),(8.26019994651558e35,-6.736984599062694e-19),(-1.4143671344485664e-238,-1.220003479858045e203),(2.466089772925698e-207,1.0025476904532926e-242),(-6.3786667153054354e240,-7.010834902137467e-103),(-6.766918514324285e-263,7.404639608483947e188),(2.753493977757937e126,-4.089565842001999e-152)],[(4.339873790493155e239,-5.022554811588342e24),(-1.7712390083519473e-66,1.3290563068463308e112),(3.3648764781548893e233,1.1123394188044336e112),(-5.415278137684864e195,5.590597851016202e-270),(-2.1032310903543943e99,-2.2335799924679948e-184)]]"
"",2396526460,"[[(1.9925796792641788e-261,1.647618305107044e158),(3.014593666207223e-222,-9.016473078578002e-20),(-1.5307802021477097e-230,-7.867078587209265e-243),(-7.330317098800564e295,1.7496539408601967e-281)],[(2.2816938730052074e98,-3.3089122320442997e-136),(-4.930983789361344e-263,-6.526758521792829e59),(-2.6482873886835413e34,-4.1985691142515947e83),(1.5496810029349365e238,-4.790553105593492e71),(-7.597436233325566e83,-1.3791763752378415e137),(-1.917321980700588e-307,-1.5913257477581824e62)]]"
"=@ep",3618088392,"[[(-2.2303235811290024e-306,8.64070367587338e-13),(-7.403012423264767e-129,-1.0825508572345856e-147),(-3.6080301450167e286,1.7302718548299961e285),(-1.3839239794870825e-156,4.255424291564323e107),(2.3191305762555e-33,-2.873899421579949e-145),(7.237414513124649e-159,-4.926574547865783e178),(4.251831312243431e-199,1.2164714479391436e201)],[(-5.114074387943793e242,2.0119340496886292e295),(-3.3663670765548e-262,-6.1992631068472835e221),(1.1539386993255106e-261,1.582903697171063e-33),(-6.1914577817088e118,-1.0401495621681123e145)],[],[(-5.9815907467493136e82,4.369047439375412e219),(-4.485368440431237e89,-3.633023372434946e-59),(-2.087497331251707e-180,1.0524018118646965e257)],[(-1.2636503461000215e-228,-4.8426877075223456e204),(2.74943107551342e281,-7.453097760262003e-14)]]"
"",3467776823,"[]"
"b'zQ",484159052,"[[(3.041838095219909e276,-6.956822159518612e-87)],[(6.636906358770296e-97,1.0531865724169307e-214)],[(-8.429249069245283e-243,-2.134779842898037e243)],[(-0.4657586598569572,2.799768548127799e187),(-5.961335445789657e-129,2.560331789344886e293),(-3.139409694983184e45,2.8011384557268085e-47)]]"
"6xGw",4126129912,"[]"
"Q",3109335413,"[[(-2.8435266267772945e39,9.548278488724291e26),(-1.1682790407223344e46,-3.925561182768867e-266),(2.8381633655721614e-202,-3.472921303086527e40),(3.3968328275944204e-150,-2.2188876184777275e-69),(-1.2612795000783405e-88,-1.2942793285205966e-49),(1.3678466236967012e179,1.721664680964459e97),(-1.1020844667744628e198,-3.403142062758506e-47)],[],[(1.343149099058239e-279,9.397894929770352e-132),(-5.280854317597215e250,9.862550191577643e-292),(-7.11468799151533e-58,7.510011657942604e96),(1.183774454157175e-288,-1.5697197095936546e272),(-3.727289017361602e120,2.422831380775067e-107),(1.4345094301262986e-177,2.4990983297605437e-91)],[(9.195226893854516e169,6.546374357272709e-236),(2.320311199531441e-126,2.2257031285964243e-185),(3.351868475505779e-184,1.84394695526876e88)],[(1.6290814396647987e-112,-3.589542711073253e38),(4.0060174859833907e-261,-1.9900431208726192e-296),(2.047468933030435e56,8.483912759156179e-57),(3.1165727272872075e191,-1.5487136748040008e-156),(0.43564020198461034,4.618165048931035e-244),(-7.674951896752824e-214,1.1652522629091777e-105),(4.838653901829244e-89,5.3085904574780206e169)],[(1.8286703553352283e-246,2.0403170465657044e255),(2.040810692623279e267,4.3956975402250484e-8),(2.4101343663018673e131,-8.672394158504762e167),(3.092080945239809e-219,-3.775474693770226e293),(-1.527991241079512e-15,-1.2603969180963007e226),(9.17470637459212e-56,1.6021090930395906e-133),(7.877647227721046e58,3.2592118033868903e-108)],[(1.4334765313272463e170,2.6971234798957105e-50)]]"
"^ip",1015254922,"[[(-2.227414144223298e-63,1.2391785738638914e276),(1.2668491759136862e207,2.5656762953078853e-67),(2.385410876813441e-268,1.451107969531624e25),(-5.475956161647574e131,2239495689376746),(1.5591286361054593e180,3.672868971445151e117)]]"
"5N]",1720727300,"[[(-2.0670321228319122e-258,-2.6893477429616666e-32),(-2.2424105705209414e225,3.547832127050775e25),(4.452916756606404e-121,-3.71114618421911e156),(-1.966961937965055e-110,3.1217044497868816e227),(20636923519704216,1.3500210618276638e30),(3.3195926701816527e-276,1.5557140338374535e234)],[]]"
1 fSRH 976027584 [[(-1.5346513608456012e-204,-2.867937504545497e266),(3.1627675144114637e-231,-2.20343471241604e-54),(-1.866886218651809e-89,-7.695893036366416e100),(8.196307577166986e-169,-8.203793887684096e-263),(-1.6150328830402252e-215,8.531116551449711e-296),(4.3378407855931477e92,1.1313645428723989e117),(-4.238081208165573e137,-8.969951719788361e67)],[(-3.409639554701108e169,-7.277093176871153e-254),(1.1466207153308928e-226,3.429893348531029e96),(6.451302850199177e-189,-7.52379443153242e125),(-1.7132078539493614e-127,-2.3177814806867505e241),(1.4996520594989919e-257,4.271017883966942e128)],[(65460976657479156000,1.7055814145588595e253),(-1.921491101580189e154,3.2912740465446566e-286),(0.0008437955075350972,-5.143493717005472e-107),(8.637208599142187e-150,7.825076274945548e136),(1.8077733932468565e-159,5.51061479974026e-77),(1.300406236793709e-260,10669142.497111017),(-1.731981751951159e91,-1.270795062098902e102)],[(3.336706342781395e-7,-1.1919528866481513e266)]]
2 sX6> 733011552 [[(-3.737863336077909e-44,3.066510481088993e-161),(-1.0047259170558555e-31,8.066145272086467e-274)],[(1.2261835328136691e-58,-6.154561379350395e258),(8.26019994651558e35,-6.736984599062694e-19),(-1.4143671344485664e-238,-1.220003479858045e203),(2.466089772925698e-207,1.0025476904532926e-242),(-6.3786667153054354e240,-7.010834902137467e-103),(-6.766918514324285e-263,7.404639608483947e188),(2.753493977757937e126,-4.089565842001999e-152)],[(4.339873790493155e239,-5.022554811588342e24),(-1.7712390083519473e-66,1.3290563068463308e112),(3.3648764781548893e233,1.1123394188044336e112),(-5.415278137684864e195,5.590597851016202e-270),(-2.1032310903543943e99,-2.2335799924679948e-184)]]
3 2396526460 [[(1.9925796792641788e-261,1.647618305107044e158),(3.014593666207223e-222,-9.016473078578002e-20),(-1.5307802021477097e-230,-7.867078587209265e-243),(-7.330317098800564e295,1.7496539408601967e-281)],[(2.2816938730052074e98,-3.3089122320442997e-136),(-4.930983789361344e-263,-6.526758521792829e59),(-2.6482873886835413e34,-4.1985691142515947e83),(1.5496810029349365e238,-4.790553105593492e71),(-7.597436233325566e83,-1.3791763752378415e137),(-1.917321980700588e-307,-1.5913257477581824e62)]]
4 =@ep 3618088392 [[(-2.2303235811290024e-306,8.64070367587338e-13),(-7.403012423264767e-129,-1.0825508572345856e-147),(-3.6080301450167e286,1.7302718548299961e285),(-1.3839239794870825e-156,4.255424291564323e107),(2.3191305762555e-33,-2.873899421579949e-145),(7.237414513124649e-159,-4.926574547865783e178),(4.251831312243431e-199,1.2164714479391436e201)],[(-5.114074387943793e242,2.0119340496886292e295),(-3.3663670765548e-262,-6.1992631068472835e221),(1.1539386993255106e-261,1.582903697171063e-33),(-6.1914577817088e118,-1.0401495621681123e145)],[],[(-5.9815907467493136e82,4.369047439375412e219),(-4.485368440431237e89,-3.633023372434946e-59),(-2.087497331251707e-180,1.0524018118646965e257)],[(-1.2636503461000215e-228,-4.8426877075223456e204),(2.74943107551342e281,-7.453097760262003e-14)]]
5 3467776823 []
6 b'zQ 484159052 [[(3.041838095219909e276,-6.956822159518612e-87)],[(6.636906358770296e-97,1.0531865724169307e-214)],[(-8.429249069245283e-243,-2.134779842898037e243)],[(-0.4657586598569572,2.799768548127799e187),(-5.961335445789657e-129,2.560331789344886e293),(-3.139409694983184e45,2.8011384557268085e-47)]]
7 6xGw 4126129912 []
8 Q 3109335413 [[(-2.8435266267772945e39,9.548278488724291e26),(-1.1682790407223344e46,-3.925561182768867e-266),(2.8381633655721614e-202,-3.472921303086527e40),(3.3968328275944204e-150,-2.2188876184777275e-69),(-1.2612795000783405e-88,-1.2942793285205966e-49),(1.3678466236967012e179,1.721664680964459e97),(-1.1020844667744628e198,-3.403142062758506e-47)],[],[(1.343149099058239e-279,9.397894929770352e-132),(-5.280854317597215e250,9.862550191577643e-292),(-7.11468799151533e-58,7.510011657942604e96),(1.183774454157175e-288,-1.5697197095936546e272),(-3.727289017361602e120,2.422831380775067e-107),(1.4345094301262986e-177,2.4990983297605437e-91)],[(9.195226893854516e169,6.546374357272709e-236),(2.320311199531441e-126,2.2257031285964243e-185),(3.351868475505779e-184,1.84394695526876e88)],[(1.6290814396647987e-112,-3.589542711073253e38),(4.0060174859833907e-261,-1.9900431208726192e-296),(2.047468933030435e56,8.483912759156179e-57),(3.1165727272872075e191,-1.5487136748040008e-156),(0.43564020198461034,4.618165048931035e-244),(-7.674951896752824e-214,1.1652522629091777e-105),(4.838653901829244e-89,5.3085904574780206e169)],[(1.8286703553352283e-246,2.0403170465657044e255),(2.040810692623279e267,4.3956975402250484e-8),(2.4101343663018673e131,-8.672394158504762e167),(3.092080945239809e-219,-3.775474693770226e293),(-1.527991241079512e-15,-1.2603969180963007e226),(9.17470637459212e-56,1.6021090930395906e-133),(7.877647227721046e58,3.2592118033868903e-108)],[(1.4334765313272463e170,2.6971234798957105e-50)]]
9 ^ip 1015254922 [[(-2.227414144223298e-63,1.2391785738638914e276),(1.2668491759136862e207,2.5656762953078853e-67),(2.385410876813441e-268,1.451107969531624e25),(-5.475956161647574e131,2239495689376746),(1.5591286361054593e180,3.672868971445151e117)]]
10 5N] 1720727300 [[(-2.0670321228319122e-258,-2.6893477429616666e-32),(-2.2424105705209414e225,3.547832127050775e25),(4.452916756606404e-121,-3.71114618421911e156),(-1.966961937965055e-110,3.1217044497868816e227),(20636923519704216,1.3500210618276638e30),(3.3195926701816527e-276,1.5557140338374535e234)],[]]

View File

@ -0,0 +1,3 @@
"b'zQ",2960084897,"[[(3.014593666207223e-222,-7.277093176871153e-254),(-1.5307802021477097e-230,3.429893348531029e96),(-7.330317098800564e295,-7.52379443153242e125),(2.2816938730052074e98,-2.3177814806867505e241),(-4.930983789361344e-263,4.271017883966942e128)],[(-2.6482873886835413e34,1.7055814145588595e253),(1.5496810029349365e238,3.2912740465446566e-286),(-7.597436233325566e83,-5.143493717005472e-107),(-1.917321980700588e-307,7.825076274945548e136)],[(-2.2303235811290024e-306,5.51061479974026e-77),(-7.403012423264767e-129,10669142.497111017),(-3.6080301450167e286,-1.270795062098902e102),(-1.3839239794870825e-156,-1.1919528866481513e266),(2.3191305762555e-33,3.066510481088993e-161),(7.237414513124649e-159,8.066145272086467e-274)],[(4.251831312243431e-199,-6.154561379350395e258),(-5.114074387943793e242,-6.736984599062694e-19),(-3.3663670765548e-262,-1.220003479858045e203),(1.1539386993255106e-261,1.0025476904532926e-242),(-6.1914577817088e118,-7.010834902137467e-103),(-5.9815907467493136e82,7.404639608483947e188),(-4.485368440431237e89,-4.089565842001999e-152)]]"
"6xGw",2107128550,"[[(-2.087497331251707e-180,-5.022554811588342e24),(-1.2636503461000215e-228,1.3290563068463308e112),(2.74943107551342e281,1.1123394188044336e112),(3.041838095219909e276,5.590597851016202e-270)],[],[(6.636906358770296e-97,-2.2335799924679948e-184),(-8.429249069245283e-243,1.647618305107044e158),(-0.4657586598569572,-9.016473078578002e-20)]]"
"Q",2713167232,"[[(-5.961335445789657e-129,-7.867078587209265e-243),(-3.139409694983184e45,1.7496539408601967e-281)],[(-2.8435266267772945e39,-3.3089122320442997e-136)]]"
1 b'zQ 2960084897 [[(3.014593666207223e-222,-7.277093176871153e-254),(-1.5307802021477097e-230,3.429893348531029e96),(-7.330317098800564e295,-7.52379443153242e125),(2.2816938730052074e98,-2.3177814806867505e241),(-4.930983789361344e-263,4.271017883966942e128)],[(-2.6482873886835413e34,1.7055814145588595e253),(1.5496810029349365e238,3.2912740465446566e-286),(-7.597436233325566e83,-5.143493717005472e-107),(-1.917321980700588e-307,7.825076274945548e136)],[(-2.2303235811290024e-306,5.51061479974026e-77),(-7.403012423264767e-129,10669142.497111017),(-3.6080301450167e286,-1.270795062098902e102),(-1.3839239794870825e-156,-1.1919528866481513e266),(2.3191305762555e-33,3.066510481088993e-161),(7.237414513124649e-159,8.066145272086467e-274)],[(4.251831312243431e-199,-6.154561379350395e258),(-5.114074387943793e242,-6.736984599062694e-19),(-3.3663670765548e-262,-1.220003479858045e203),(1.1539386993255106e-261,1.0025476904532926e-242),(-6.1914577817088e118,-7.010834902137467e-103),(-5.9815907467493136e82,7.404639608483947e188),(-4.485368440431237e89,-4.089565842001999e-152)]]
2 6xGw 2107128550 [[(-2.087497331251707e-180,-5.022554811588342e24),(-1.2636503461000215e-228,1.3290563068463308e112),(2.74943107551342e281,1.1123394188044336e112),(3.041838095219909e276,5.590597851016202e-270)],[],[(6.636906358770296e-97,-2.2335799924679948e-184),(-8.429249069245283e-243,1.647618305107044e158),(-0.4657586598569572,-9.016473078578002e-20)]]
3 Q 2713167232 [[(-5.961335445789657e-129,-7.867078587209265e-243),(-3.139409694983184e45,1.7496539408601967e-281)],[(-2.8435266267772945e39,-3.3089122320442997e-136)]]

View File

@ -0,0 +1,5 @@
"~m`",820408404,"[]"
"~E",3621610983,"[[(1.183772215004139e-238,-1.282774073199881e211),(1.6787305112393978e-46,7.500499989257719e25),(-2.458759475104641e-260,3.1724599388651864e-171),(-2.0163203163062471e118,-4.677226438945462e-162),(-5.52491070012707e-135,7.051780441780731e-236)]]"
"~1",1715555780,"[[(-6.847404226505131e-267,5.939552045362479e-272),(8.02275075985457e-160,8.369250185716419e-104),(-1.193940928527857e-258,-1.132580458849774e39)],[(1.1866087552639048e253,3.104988412734545e57),(-3.37278669639914e84,-2.387628643569968e287),(-2.452136349495753e73,3.194309776006896e-204),(-1001997440265471100,3.482122851077378e-182)],[],[(-5.754682082202988e-20,6.598766936241908e156)],[(8.386764833095757e300,1.2049637765877942e229),(3.136243074210055e53,5.764669663844127e-100),(-4.190632347661851e195,-5.053553379163823e302),(2.0805194731736336e-19,-1.0849036699112485e-271),(1.1292361211411365e227,-8.767824448179629e229),(-3.6938137156625264e-19,-5.387931698392423e109),(-1.2240482125885677e189,-1.5631467861525635e-103)],[(-2.3917431782202442e138,7.817228281030191e-242),(-1.1462343232899826e279,-1.971215065504208e-225),(5.4316119855340265e-62,3.761081156597423e-60),(8.111852137718836e306,8.115485489580134e-208)],[]]"
"~%]",1606443384,"[[]]"
"}or",726681547,"[]"
1 ~m` 820408404 []
2 ~E 3621610983 [[(1.183772215004139e-238,-1.282774073199881e211),(1.6787305112393978e-46,7.500499989257719e25),(-2.458759475104641e-260,3.1724599388651864e-171),(-2.0163203163062471e118,-4.677226438945462e-162),(-5.52491070012707e-135,7.051780441780731e-236)]]
3 ~1 1715555780 [[(-6.847404226505131e-267,5.939552045362479e-272),(8.02275075985457e-160,8.369250185716419e-104),(-1.193940928527857e-258,-1.132580458849774e39)],[(1.1866087552639048e253,3.104988412734545e57),(-3.37278669639914e84,-2.387628643569968e287),(-2.452136349495753e73,3.194309776006896e-204),(-1001997440265471100,3.482122851077378e-182)],[],[(-5.754682082202988e-20,6.598766936241908e156)],[(8.386764833095757e300,1.2049637765877942e229),(3.136243074210055e53,5.764669663844127e-100),(-4.190632347661851e195,-5.053553379163823e302),(2.0805194731736336e-19,-1.0849036699112485e-271),(1.1292361211411365e227,-8.767824448179629e229),(-3.6938137156625264e-19,-5.387931698392423e109),(-1.2240482125885677e189,-1.5631467861525635e-103)],[(-2.3917431782202442e138,7.817228281030191e-242),(-1.1462343232899826e279,-1.971215065504208e-225),(5.4316119855340265e-62,3.761081156597423e-60),(8.111852137718836e306,8.115485489580134e-208)],[]]
4 ~%] 1606443384 [[]]
5 }or 726681547 []

View File

@ -0,0 +1,7 @@
"kvUES",4281162618,"[[(2.4538308454074088e303,1.2209370543175666e178),(1.4564007891121754e-186,2.340773478952682e-273),(-1.01791181533976e165,-3.9617466227377253e248)]]"
"Gu",4280623186,"[[(-1.623487579335014e38,-1.0633405021023563e225),(-4.373688812751571e180,2.5511550357717127e138)]]"
"J_u1",4277430503,"[[(2.981826196369429e-294,-6.059236590410922e236),(8.502045137575854e-296,3.0210403188125657e-91),(-9.370591842861745e175,4.150870185764185e129),(1.011801592194125e275,-9.236010982686472e266),(-3.1830638196303316e277,2.417706446545472e-105),(-1.4369143023804266e-201,4.7529126795899655e238)],[(-2.118789593804697e186,-1.8760231612433755e-280),(2.5982563179976053e200,-1.4683025762313524e-40)],[(-1.873397623255704e-240,1.4363190147949886e-283),(-1.5760337746177136e153,1.5272278536086246e-34),(-8.117473317695919e155,2.4375370926733504e150),(-1.179230972881795e99,1.7693459774706515e-259),(2.2102106250558424e-40,4.734162675762768e-56),(6.058833110550111e-8,8.892471775821198e164),(-1.8208740799996599e59,6.446958261080721e178)]]"
"s:\",4265055390,"[[(-3.291651377214531e-167,3.9198636942402856e185),(2.4897781692770126e176,2.579309759138358e188),(4.653945381397663e205,3.216314556208208e158),(-5.3373279440714224e-39,2.404386813826413e212),(-1.4217294382527138e307,8.874978978402512e-173)],[(8.527603121149904e-58,-5.0520795335878225e88),(-0.00022870878520550814,-3.2334214176860943e-68),(-6.97683613433404e304,-2.1573757788072144e-82),(-1.1394163455875937e36,-3.817990182461824e271),(2.4099027412881423e-209,8.542179392011098e-156),(3.2610511540394803e174,1.1692631657517616e-20)],[(3.625474290538107e261,-5.359205062039837e-193),(-3.574126569378072e-112,-5.421804160994412e265),(-4.873653931207849e-76,3219678918284.317),(-7.030770825898911e-57,1.4647389742249787e-274),(-4.4882439220492357e-203,6.569338333730439e-38)],[(-2.2418056002374865e-136,5.113251922954469e-16),(2.5156744571032497e297,-3.0536957683846124e-192)],[(1.861112291954516e306,-1.8160882143331256e129),(1.982573454900027e290,-2.451412311394593e170)],[(-2.8292230178712157e-18,1.2570198161962067e216),(6.24832495972797e-164,-2.0770908334330718e-273)],[(980143647.1858811,1.2738714961511727e106),(6.516450532397311e-184,4.088688742052062e31),(-2.246311532913914e269,-7.418103885850518e-179),(1.2222973942835046e-289,2.750544834553288e-46),(9.503169349701076e159,-1.355457053256579e215)]]"
":hzO",4263726959,"[[(-2.553206398375626e-90,1.6536977728640226e199),(1.5630078027143848e-36,2.805242683101373e-211),(2.2573933085983554e-92,3.450501333524858e292),(-1.215900901292646e-275,-3.860558658606121e272),(6.65716072773856e-145,2.5359010031217893e217)],[(-1.3308039625779135e308,1.7464622720773261e258),(-3.2986890093446374e179,3.9038871583175653e-69),(-4.3594764087383885e-95,4.229921973278908e-123),(-5.455694205415656e137,3.597894902167716e108),(1.2480860990110662e-29,-1.4873488392480292e-185),(7.563210285835444e55,-5624068447.488605)],[(3.9517937289943195e181,-3.2799189227094424e-68),(8.906762198487649e-167,3.952452177941537e-159)]]"
"a",4258301804,"[[(5.827965576703262e-281,2.2523852665173977e90)],[(-6.837604072282348e-97,8.125864241406046e-61)],[(-2.3047912084435663e53,-8.814499720685194e36),(1.2072558137199047e-79,1.2096862541827071e142),(2.2000026293774143e275,-3.2571689055108606e-199),(1.1822278574921316e134,2.9571188365006754e-86),(1.0448954272555034e-169,1.2182183489600953e-60)],[(-3.1366540817730525e89,9.327128058982966e-306),(6.588968210928936e73,-11533531378.938957),(-2.6715943555840563e44,-4.557428011172859e224),(-3.8334913754415923e285,-4.748721454106074e-173),(-1.6912052107425128e275,-4.789382438422238e-219),(1.8538365229016863e151,-3.5698172075468775e-37)],[(-2.1963131282037294e49,-5.53604352524995e-296)],[(-8.834414834987965e167,1.3186354307320576e247),(2.109209547987338e298,1.2191009105107557e-32),(-3.896880410603213e-92,-3.4589588698231044e-121),(-3.252529090888335e138,-7.862741341454407e204)],[(-9.673078095447289e-207,8.839303128607278e123),(2.6043620378793597e-244,-6.898328199987363e-308),(-2.5921142292355475e-54,1.0352159149517285e-143)]]"
"S+",4257734123,"[[(1.5714269203495863e245,-15651321.549208183),(-3.7292056272445236e-254,-4.556927533596056e-234),(-3.0309414401442555e-203,-3.84393827531526e-12)],[(1.7718777510571518e219,3.972086323144777e139),(1.5723805735454373e-67,-3.805243648123396e226),(154531069271292800000,1.1384408025183933e-285),(-2.009892367470994e-247,2.0325742976832167e81)],[(1.2145787097670788e55,-5.0579298233321666e-30),(5.05577441452021e-182,-2.968914705509665e-175),(-1.702335524921919e67,-2.852552828587631e-226),(-2.7664498327826963e-99,-1.2967072085088717e-305),(7.68881162387673e-68,-1.2506915095983359e-142),(-7.60308693295946e-40,5.414853590549086e218)],[(8.595602987813848e226,-3.9708286611967497e-206),(-5.80352787694746e-52,5.610493934761672e236),(2.1336999375861025e217,-5.431988994371099e-154),(-6.2758614367782974e29,-8.359901046980544e-55)],[(1.6910790690897504e54,9.798739710823911e197),(-6.530270107036228e-284,8.758552462406328e-302),(2.931625032390877e-118,2.8793800873550273e83),(-3.293986884112906e-88,11877326093331202),(0.0008071321465157103,1.0720860516457485e-298)]]"
1 kvUES 4281162618 [[(2.4538308454074088e303,1.2209370543175666e178),(1.4564007891121754e-186,2.340773478952682e-273),(-1.01791181533976e165,-3.9617466227377253e248)]]
2 Gu 4280623186 [[(-1.623487579335014e38,-1.0633405021023563e225),(-4.373688812751571e180,2.5511550357717127e138)]]
3 J_u1 4277430503 [[(2.981826196369429e-294,-6.059236590410922e236),(8.502045137575854e-296,3.0210403188125657e-91),(-9.370591842861745e175,4.150870185764185e129),(1.011801592194125e275,-9.236010982686472e266),(-3.1830638196303316e277,2.417706446545472e-105),(-1.4369143023804266e-201,4.7529126795899655e238)],[(-2.118789593804697e186,-1.8760231612433755e-280),(2.5982563179976053e200,-1.4683025762313524e-40)],[(-1.873397623255704e-240,1.4363190147949886e-283),(-1.5760337746177136e153,1.5272278536086246e-34),(-8.117473317695919e155,2.4375370926733504e150),(-1.179230972881795e99,1.7693459774706515e-259),(2.2102106250558424e-40,4.734162675762768e-56),(6.058833110550111e-8,8.892471775821198e164),(-1.8208740799996599e59,6.446958261080721e178)]]
4 s:\ 4265055390 [[(-3.291651377214531e-167,3.9198636942402856e185),(2.4897781692770126e176,2.579309759138358e188),(4.653945381397663e205,3.216314556208208e158),(-5.3373279440714224e-39,2.404386813826413e212),(-1.4217294382527138e307,8.874978978402512e-173)],[(8.527603121149904e-58,-5.0520795335878225e88),(-0.00022870878520550814,-3.2334214176860943e-68),(-6.97683613433404e304,-2.1573757788072144e-82),(-1.1394163455875937e36,-3.817990182461824e271),(2.4099027412881423e-209,8.542179392011098e-156),(3.2610511540394803e174,1.1692631657517616e-20)],[(3.625474290538107e261,-5.359205062039837e-193),(-3.574126569378072e-112,-5.421804160994412e265),(-4.873653931207849e-76,3219678918284.317),(-7.030770825898911e-57,1.4647389742249787e-274),(-4.4882439220492357e-203,6.569338333730439e-38)],[(-2.2418056002374865e-136,5.113251922954469e-16),(2.5156744571032497e297,-3.0536957683846124e-192)],[(1.861112291954516e306,-1.8160882143331256e129),(1.982573454900027e290,-2.451412311394593e170)],[(-2.8292230178712157e-18,1.2570198161962067e216),(6.24832495972797e-164,-2.0770908334330718e-273)],[(980143647.1858811,1.2738714961511727e106),(6.516450532397311e-184,4.088688742052062e31),(-2.246311532913914e269,-7.418103885850518e-179),(1.2222973942835046e-289,2.750544834553288e-46),(9.503169349701076e159,-1.355457053256579e215)]]
5 :hzO 4263726959 [[(-2.553206398375626e-90,1.6536977728640226e199),(1.5630078027143848e-36,2.805242683101373e-211),(2.2573933085983554e-92,3.450501333524858e292),(-1.215900901292646e-275,-3.860558658606121e272),(6.65716072773856e-145,2.5359010031217893e217)],[(-1.3308039625779135e308,1.7464622720773261e258),(-3.2986890093446374e179,3.9038871583175653e-69),(-4.3594764087383885e-95,4.229921973278908e-123),(-5.455694205415656e137,3.597894902167716e108),(1.2480860990110662e-29,-1.4873488392480292e-185),(7.563210285835444e55,-5624068447.488605)],[(3.9517937289943195e181,-3.2799189227094424e-68),(8.906762198487649e-167,3.952452177941537e-159)]]
6 a 4258301804 [[(5.827965576703262e-281,2.2523852665173977e90)],[(-6.837604072282348e-97,8.125864241406046e-61)],[(-2.3047912084435663e53,-8.814499720685194e36),(1.2072558137199047e-79,1.2096862541827071e142),(2.2000026293774143e275,-3.2571689055108606e-199),(1.1822278574921316e134,2.9571188365006754e-86),(1.0448954272555034e-169,1.2182183489600953e-60)],[(-3.1366540817730525e89,9.327128058982966e-306),(6.588968210928936e73,-11533531378.938957),(-2.6715943555840563e44,-4.557428011172859e224),(-3.8334913754415923e285,-4.748721454106074e-173),(-1.6912052107425128e275,-4.789382438422238e-219),(1.8538365229016863e151,-3.5698172075468775e-37)],[(-2.1963131282037294e49,-5.53604352524995e-296)],[(-8.834414834987965e167,1.3186354307320576e247),(2.109209547987338e298,1.2191009105107557e-32),(-3.896880410603213e-92,-3.4589588698231044e-121),(-3.252529090888335e138,-7.862741341454407e204)],[(-9.673078095447289e-207,8.839303128607278e123),(2.6043620378793597e-244,-6.898328199987363e-308),(-2.5921142292355475e-54,1.0352159149517285e-143)]]
7 S+ 4257734123 [[(1.5714269203495863e245,-15651321.549208183),(-3.7292056272445236e-254,-4.556927533596056e-234),(-3.0309414401442555e-203,-3.84393827531526e-12)],[(1.7718777510571518e219,3.972086323144777e139),(1.5723805735454373e-67,-3.805243648123396e226),(154531069271292800000,1.1384408025183933e-285),(-2.009892367470994e-247,2.0325742976832167e81)],[(1.2145787097670788e55,-5.0579298233321666e-30),(5.05577441452021e-182,-2.968914705509665e-175),(-1.702335524921919e67,-2.852552828587631e-226),(-2.7664498327826963e-99,-1.2967072085088717e-305),(7.68881162387673e-68,-1.2506915095983359e-142),(-7.60308693295946e-40,5.414853590549086e218)],[(8.595602987813848e226,-3.9708286611967497e-206),(-5.80352787694746e-52,5.610493934761672e236),(2.1336999375861025e217,-5.431988994371099e-154),(-6.2758614367782974e29,-8.359901046980544e-55)],[(1.6910790690897504e54,9.798739710823911e197),(-6.530270107036228e-284,8.758552462406328e-302),(2.931625032390877e-118,2.8793800873550273e83),(-3.293986884112906e-88,11877326093331202),(0.0008071321465157103,1.0720860516457485e-298)]]

View File

@ -0,0 +1,129 @@
import logging
import os
import pytest
from helpers.cluster import ClickHouseCluster
from helpers.test_tools import TSV
logging.getLogger().setLevel(logging.INFO)
logging.getLogger().addHandler(logging.StreamHandler())
SCRIPT_DIR = os.path.dirname(os.path.realpath(__file__))
S3_DATA = ['data/clickhouse/part1.csv', 'data/clickhouse/part123.csv', 'data/database/part2.csv', 'data/database/partition675.csv']
def create_buckets_s3(cluster):
minio = cluster.minio_client
for file in S3_DATA:
minio.fput_object(bucket_name=cluster.minio_bucket, object_name=file, file_path=os.path.join(SCRIPT_DIR, file))
for obj in minio.list_objects(cluster.minio_bucket, recursive=True):
print(obj.object_name)
@pytest.fixture(scope="module")
def started_cluster():
try:
cluster = ClickHouseCluster(__file__)
cluster.add_instance('s0_0_0', main_configs=["configs/cluster.xml"], with_minio=True)
cluster.add_instance('s0_0_1', main_configs=["configs/cluster.xml"])
cluster.add_instance('s0_1_0', main_configs=["configs/cluster.xml"])
logging.info("Starting cluster...")
cluster.start()
logging.info("Cluster started")
create_buckets_s3(cluster)
yield cluster
finally:
cluster.shutdown()
def test_select_all(started_cluster):
node = started_cluster.instances['s0_0_0']
pure_s3 = node.query("""
SELECT * from s3(
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
ORDER BY (name, value, polygon)""")
# print(pure_s3)
s3_distibuted = node.query("""
SELECT * from s3Cluster(
'cluster_simple',
'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))') ORDER BY (name, value, polygon)""")
# print(s3_distibuted)
assert TSV(pure_s3) == TSV(s3_distibuted)
def test_count(started_cluster):
node = started_cluster.instances['s0_0_0']
pure_s3 = node.query("""
SELECT count(*) from s3(
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')""")
# print(pure_s3)
s3_distibuted = node.query("""
SELECT count(*) from s3Cluster(
'cluster_simple', 'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')""")
# print(s3_distibuted)
assert TSV(pure_s3) == TSV(s3_distibuted)
def test_union_all(started_cluster):
node = started_cluster.instances['s0_0_0']
pure_s3 = node.query("""
SELECT * FROM
(
SELECT * from s3(
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
UNION ALL
SELECT * from s3(
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
)
ORDER BY (name, value, polygon)
""")
# print(pure_s3)
s3_distibuted = node.query("""
SELECT * FROM
(
SELECT * from s3Cluster(
'cluster_simple',
'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
UNION ALL
SELECT * from s3Cluster(
'cluster_simple',
'http://minio1:9001/root/data/{clickhouse,database}/*', 'minio', 'minio123', 'CSV',
'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
)
ORDER BY (name, value, polygon)
""")
# print(s3_distibuted)
assert TSV(pure_s3) == TSV(s3_distibuted)
def test_wrong_cluster(started_cluster):
node = started_cluster.instances['s0_0_0']
error = node.query_and_get_error("""
SELECT count(*) from s3Cluster(
'non_existent_cluster',
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')
UNION ALL
SELECT count(*) from s3Cluster(
'non_existent_cluster',
'http://minio1:9001/root/data/{clickhouse,database}/*',
'minio', 'minio123', 'CSV', 'name String, value UInt32, polygon Array(Array(Tuple(Float64, Float64)))')""")
assert "not found" in error

View File

@ -0,0 +1,12 @@
#!/usr/bin/env bash
# NOTE: this is a partial copy of the 01683_dist_INSERT_block_structure_mismatch,
# but this test also checks the log messages
CUR_DIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
# shellcheck source=../shell_config.sh
. "$CUR_DIR"/../shell_config.sh
${CLICKHOUSE_CLIENT_BINARY} --send_logs_level="none" -q "SELECT * FROM s3('https://s3.mds.yandex.net/clickhouse-test-reports/*/*/functional_stateless_tests_(ubsan)/test_results.tsv', '$S3_ACCESS_KEY_ID', '$S3_SECRET_ACCESS', 'LineAsString', 'line String') limit 100 FORMAT Null;"
${CLICKHOUSE_CLIENT_BINARY} --send_logs_level="none" -q "SELECT * FROM s3Cluster('test_cluster_two_shards', 'https://s3.mds.yandex.net/clickhouse-test-reports/*/*/functional_stateless_tests_(ubsan)/test_results.tsv', '$S3_ACCESS_KEY_ID', '$S3_SECRET_ACCESS', 'LineAsString', 'line String') limit 100 FORMAT Null;"

View File

@ -229,3 +229,4 @@
01791_dist_INSERT_block_structure_mismatch 01791_dist_INSERT_block_structure_mismatch
01801_distinct_group_by_shard 01801_distinct_group_by_shard
01804_dictionary_decimal256_type 01804_dictionary_decimal256_type
01801_s3_distributed