ClickHouse/src/Storages/StorageS3Cluster.cpp

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

115 lines
4.1 KiB
C++
Raw Normal View History

2021-04-12 21:42:52 +00:00
#include "Storages/StorageS3Cluster.h"
2021-03-19 21:49:18 +00:00
2021-03-24 21:02:21 +00:00
#if USE_AWS_S3
2021-03-22 17:12:31 +00:00
#include <DataTypes/DataTypeString.h>
Fix terribly broken, fragile and potentially cyclic linking Sorry for the clickbaity title. This is about static method ConnectionTimeouts::getHTTPTimeouts(). It was be declared in header IO/ConnectionTimeouts.h, and defined in header IO/ConnectionTimeoutsContext.h (!). This is weird and caused issues with linking on s390x (##45520). There was an attempt to fix some inconsistencies (#45848) but neither did @Algunenano nor me at first really understand why the definition is in the header. Turns out that ConnectionTimeoutsContext.h is only #include'd from source files which are part of the normal server build BUT NOT part of the keeper standalone build (which must be enabled via CMake -DBUILD_STANDALONE_KEEPER=1). This dependency was not documented and as a result, some misguided workarounds were introduced earlier, e.g. https://github.com/ClickHouse/ClickHouse/pull/38475/commits/0341c6c54bd7ac77200b4ca123208b195514ef20 The deeper cause was that getHTTPTimeouts() is passed a "Context". This class is part of the "dbms" libary which is deliberately not linked by the standalone build of clickhouse-keeper. The context is only used to read the settings and the "Settings" class is part of the clickhouse_common library which is linked by clickhouse-keeper already. To resolve this mess, this PR - creates source file IO/ConnectionTimeouts.cpp and moves all ConnectionTimeouts definitions into it, including getHTTPTimeouts(). - breaks the wrong dependency by passing "Settings" instead of "Context" into getHTTPTimeouts(). - resolves the previous hacks
2023-02-03 10:54:49 +00:00
#include <IO/ConnectionTimeouts.h>
#include <Interpreters/AddDefaultDatabaseVisitor.h>
#include <Interpreters/InterpreterSelectQuery.h>
2021-04-06 19:18:45 +00:00
#include <Processors/Sources/RemoteSource.h>
#include <Processors/Transforms/AddingDefaultsTransform.h>
2021-10-15 20:18:20 +00:00
#include <QueryPipeline/RemoteQueryExecutor.h>
2021-03-26 15:33:14 +00:00
#include <Storages/IStorage.h>
#include <Storages/StorageURL.h>
2021-03-26 15:33:14 +00:00
#include <Storages/SelectQueryInfo.h>
#include <Storages/StorageDictionary.h>
#include <Storages/extractTableFunctionArgumentsFromSelectQuery.h>
#include <Storages/VirtualColumnUtils.h>
#include <Common/Exception.h>
#include <Parsers/queryToString.h>
#include <TableFunctions/TableFunctionS3Cluster.h>
2021-03-22 17:12:31 +00:00
2021-03-19 21:49:18 +00:00
#include <memory>
#include <string>
2021-03-22 17:12:31 +00:00
2021-03-19 21:49:18 +00:00
namespace DB
{
2023-05-15 16:39:26 +00:00
namespace ErrorCodes
{
extern const int LOGICAL_ERROR;
}
2021-04-12 21:42:52 +00:00
StorageS3Cluster::StorageS3Cluster(
const String & cluster_name_,
const StorageS3::Configuration & configuration_,
2021-03-22 17:12:31 +00:00
const StorageID & table_id_,
const ColumnsDescription & columns_,
const ConstraintsDescription & constraints_,
const ContextPtr & context)
: IStorageCluster(cluster_name_, table_id_, getLogger("StorageS3Cluster (" + table_id_.table_name + ")"))
, s3_configuration{configuration_}
2021-03-19 21:49:18 +00:00
{
context->getGlobalContext()->getRemoteHostFilter().checkURL(configuration_.url.uri);
context->getGlobalContext()->getHTTPHeaderFilter().checkHeaders(configuration_.headers_from_ast);
2023-06-15 13:49:49 +00:00
2021-03-19 21:49:18 +00:00
StorageInMemoryMetadata storage_metadata;
updateConfigurationIfChanged(context);
if (columns_.empty())
{
ColumnsDescription columns;
/// `format_settings` is set to std::nullopt, because StorageS3Cluster is used only as table function
if (s3_configuration.format == "auto")
std::tie(columns, s3_configuration.format) = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context);
else
columns = StorageS3::getTableStructureFromData(s3_configuration, /*format_settings=*/std::nullopt, context);
storage_metadata.setColumns(columns);
}
else
{
if (s3_configuration.format == "auto")
s3_configuration.format = StorageS3::getTableStructureAndFormatFromData(s3_configuration, /*format_settings=*/std::nullopt, context).second;
storage_metadata.setColumns(columns_);
}
2021-03-22 17:12:31 +00:00
storage_metadata.setConstraints(constraints_);
2021-03-19 21:49:18 +00:00
setInMemoryMetadata(storage_metadata);
2022-05-19 11:18:58 +00:00
virtual_columns = VirtualColumnUtils::getPathFileAndSizeVirtualsForStorage(storage_metadata.getSampleBlock().getNamesAndTypesList());
2021-03-19 21:49:18 +00:00
}
void StorageS3Cluster::updateQueryToSendIfNeeded(DB::ASTPtr & query, const DB::StorageSnapshotPtr & storage_snapshot, const DB::ContextPtr & context)
{
ASTExpressionList * expression_list = extractTableFunctionArgumentsFromSelectQuery(query);
if (!expression_list)
throw Exception(ErrorCodes::LOGICAL_ERROR, "Expected SELECT query from table function s3Cluster, got '{}'", queryToString(query));
TableFunctionS3Cluster::updateStructureAndFormatArgumentsIfNeeded(
expression_list->children,
storage_snapshot->metadata->getColumns().getAll().toNamesAndTypesDescription(),
s3_configuration.format,
context);
}
2023-03-28 13:39:59 +00:00
void StorageS3Cluster::updateConfigurationIfChanged(ContextPtr local_context)
{
s3_configuration.update(local_context);
}
RemoteQueryExecutor::Extension StorageS3Cluster::getTaskIteratorExtension(const ActionsDAG::Node * predicate, const ContextPtr & context) const
{
auto iterator = std::make_shared<StorageS3Source::DisclosedGlobIterator>(
*s3_configuration.client, s3_configuration.url, predicate, virtual_columns, context, nullptr, s3_configuration.request_settings, context->getFileProgressCallback());
2023-09-28 14:25:04 +00:00
2023-09-28 11:51:07 +00:00
auto callback = std::make_shared<std::function<String()>>([iterator]() mutable -> String
{
if (auto next = iterator->next())
return next->key;
return "";
});
return RemoteQueryExecutor::Extension{ .task_iterator = std::move(callback) };
}
2021-04-12 21:42:52 +00:00
NamesAndTypesList StorageS3Cluster::getVirtuals() const
2021-04-12 17:33:55 +00:00
{
2022-05-19 11:18:58 +00:00
return virtual_columns;
2021-04-12 17:33:55 +00:00
}
2021-03-19 21:49:18 +00:00
}
2021-03-24 21:02:21 +00:00
#endif