ClickHouse/src/Storages/StorageXDBC.cpp

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

210 lines
6.7 KiB
C++
Raw Normal View History

#include <Storages/StorageXDBC.h>
#include <Storages/StorageFactory.h>
#include <Storages/StorageURL.h>
#include <Storages/transformQueryForExternalDatabase.h>
#include <Storages/checkAndGetLiteralArgument.h>
#include <Formats/FormatFactory.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>
2018-09-25 14:29:44 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h>
#include <Poco/Net/HTTPRequest.h>
2021-10-16 14:03:50 +00:00
#include <QueryPipeline/Pipe.h>
2022-04-27 15:05:45 +00:00
#include <Common/logger_useful.h>
2021-04-17 08:09:22 +00:00
#include <Common/escapeForFileName.h>
2018-09-25 14:29:44 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
2018-09-25 14:29:44 +00:00
StorageXDBC::StorageXDBC(
2019-12-04 16:06:55 +00:00
const StorageID & table_id_,
const std::string & remote_database_name_,
const std::string & remote_table_name_,
ColumnsDescription columns_,
ConstraintsDescription constraints_,
2021-04-23 12:18:23 +00:00
const String & comment,
ContextPtr context_,
const BridgeHelperPtr bridge_helper_)
2021-04-23 12:18:23 +00:00
: IStorageURLBase(
2021-10-26 09:31:01 +00:00
"",
2021-04-23 12:18:23 +00:00
context_,
table_id_,
IXDBCBridgeHelper::DEFAULT_FORMAT,
getFormatSettings(context_),
2022-08-21 13:53:48 +00:00
columns_,
constraints_,
2021-04-23 12:18:23 +00:00
comment,
"" /* CompressionMethod */)
, bridge_helper(bridge_helper_)
, remote_database_name(remote_database_name_)
, remote_table_name(remote_table_name_)
, log(&Poco::Logger::get("Storage" + bridge_helper->getName()))
{
2021-11-01 09:52:27 +00:00
uri = bridge_helper->getMainURI().toString();
}
2018-09-25 14:29:44 +00:00
std::string StorageXDBC::getReadMethod() const
{
return Poco::Net::HTTPRequest::HTTP_POST;
}
2018-09-25 14:29:44 +00:00
std::vector<std::pair<std::string, std::string>> StorageXDBC::getReadURIParams(
2021-04-17 08:09:22 +00:00
const Names & /* column_names */,
const StorageSnapshotPtr & /*storage_snapshot*/,
const SelectQueryInfo & /*query_info*/,
ContextPtr /*context*/,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t max_block_size) const
{
2021-04-17 08:09:22 +00:00
return bridge_helper->getURLParams(max_block_size);
}
2018-09-25 14:29:44 +00:00
std::function<void(std::ostream &)> StorageXDBC::getReadPOSTDataCallback(
2021-04-17 08:09:22 +00:00
const Names & column_names,
2022-02-28 13:29:05 +00:00
const ColumnsDescription & columns_description,
const SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum & /*processed_stage*/,
size_t /*max_block_size*/) const
{
String query = transformQueryForExternalDatabase(
query_info,
column_names,
2022-02-28 13:29:05 +00:00
columns_description.getOrdinary(),
bridge_helper->getIdentifierQuotingStyle(),
remote_database_name,
remote_table_name,
local_context);
LOG_TRACE(log, "Query: {}", query);
2021-04-17 08:09:22 +00:00
NamesAndTypesList cols;
for (const String & name : column_names)
{
2022-02-28 13:29:05 +00:00
auto column_data = columns_description.getPhysical(name);
2021-04-17 08:09:22 +00:00
cols.emplace_back(column_data.name, column_data.type);
}
auto write_body_callback = [query, cols](std::ostream & os)
{
os << "sample_block=" << escapeForFileName(cols.toString());
os << "&";
os << "query=" << escapeForFileName(query);
};
return write_body_callback;
}
2018-09-25 14:29:44 +00:00
2020-08-03 13:54:14 +00:00
Pipe StorageXDBC::read(
const Names & column_names,
const StorageSnapshotPtr & storage_snapshot,
SelectQueryInfo & query_info,
ContextPtr local_context,
QueryProcessingStage::Enum processed_stage,
size_t max_block_size,
size_t num_streams)
{
storage_snapshot->check(column_names);
2018-09-25 14:29:44 +00:00
bridge_helper->startBridgeSync();
return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
}
2018-09-25 14:29:44 +00:00
2022-02-28 13:29:05 +00:00
SinkToStoragePtr StorageXDBC::write(const ASTPtr & /* query */, const StorageMetadataPtr & metadata_snapshot, ContextPtr local_context)
2020-05-05 23:42:44 +00:00
{
2020-04-28 00:56:44 +00:00
bridge_helper->startBridgeSync();
2021-11-01 09:52:27 +00:00
auto request_uri = Poco::URI(uri);
2020-04-28 00:56:44 +00:00
request_uri.setPath("/write");
2021-04-17 08:09:22 +00:00
auto url_params = bridge_helper->getURLParams(65536);
2020-04-28 00:56:44 +00:00
for (const auto & [param, value] : url_params)
request_uri.addQueryParameter(param, value);
2021-04-17 08:09:22 +00:00
2020-04-28 00:56:44 +00:00
request_uri.addQueryParameter("db_name", remote_database_name);
request_uri.addQueryParameter("table_name", remote_table_name);
2020-05-05 23:42:44 +00:00
request_uri.addQueryParameter("format_name", format_name);
2021-04-17 08:09:22 +00:00
request_uri.addQueryParameter("sample_block", metadata_snapshot->getSampleBlock().getNamesAndTypesList().toString());
2020-04-28 00:56:44 +00:00
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
2021-07-23 14:25:35 +00:00
return std::make_shared<StorageURLSink>(
2021-10-26 09:31:01 +00:00
request_uri.toString(),
format_name,
getFormatSettings(local_context),
metadata_snapshot->getSampleBlock(),
local_context,
2023-02-07 12:10:26 +00:00
ConnectionTimeouts::getHTTPTimeouts(
local_context->getSettingsRef(),
{local_context->getConfigRef().getUInt("keep_alive_timeout", DEFAULT_HTTP_KEEP_ALIVE_TIMEOUT), 0}),
compression_method);
2020-04-28 00:56:44 +00:00
}
2018-09-25 14:29:44 +00:00
2022-05-13 18:39:19 +00:00
bool StorageXDBC::supportsSubsetOfColumns() const
{
2022-02-28 13:29:05 +00:00
return true;
}
Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const
{
return storage_snapshot->getSampleBlockForColumns(column_names);
}
std::string StorageXDBC::getName() const
{
return bridge_helper->getName();
}
namespace
{
template <typename BridgeHelperMixin>
void registerXDBCStorage(StorageFactory & factory, const std::string & name)
{
factory.registerStorage(name, [name](const StorageFactory::Arguments & args)
2018-10-10 08:38:54 +00:00
{
ASTs & engine_args = args.engine_args;
if (engine_args.size() != 3)
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage {} requires exactly 3 parameters: {}('DSN', database or schema, table)", name, name);
for (size_t i = 0; i < 3; ++i)
engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.getLocalContext());
BridgeHelperPtr bridge_helper = std::make_shared<XDBCBridgeHelper<BridgeHelperMixin>>(args.getContext(),
args.getContext()->getSettingsRef().http_receive_timeout.value,
checkAndGetLiteralArgument<String>(engine_args[0], "connection_string"),
args.getContext()->getSettingsRef().odbc_bridge_use_connection_pooling.value);
2021-04-23 12:18:23 +00:00
return std::make_shared<StorageXDBC>(
args.table_id,
checkAndGetLiteralArgument<String>(engine_args[1], "database_name"),
checkAndGetLiteralArgument<String>(engine_args[2], "table_name"),
args.columns,
args.constraints,
2021-04-23 12:18:23 +00:00
args.comment,
args.getContext(),
bridge_helper);
},
{
.source_access_type = BridgeHelperMixin::getSourceAccessType(),
});
}
}
void registerStorageJDBC(StorageFactory & factory)
{
registerXDBCStorage<JDBCBridgeMixin>(factory, "JDBC");
}
2018-09-25 14:29:44 +00:00
void registerStorageODBC(StorageFactory & factory)
{
registerXDBCStorage<ODBCBridgeMixin>(factory, "ODBC");
2018-09-25 14:29:44 +00:00
}
2018-10-10 08:38:54 +00:00
}