2022-06-23 20:04:06 +00:00
|
|
|
#include <Storages/StorageXDBC.h>
|
|
|
|
#include <Storages/StorageFactory.h>
|
|
|
|
#include <Storages/StorageURL.h>
|
|
|
|
#include <Storages/transformQueryForExternalDatabase.h>
|
|
|
|
#include <Storages/checkAndGetLiteralArgument.h>
|
2020-11-05 11:28:20 +00:00
|
|
|
|
|
|
|
#include <Formats/FormatFactory.h>
|
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>
|
2020-11-05 11:28:20 +00:00
|
|
|
#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>
|
|
|
|
|
2020-02-17 15:33:16 +00:00
|
|
|
|
2018-09-25 14:29:44 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
2018-10-03 10:44:43 +00:00
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
|
|
|
|
2019-07-09 15:40:21 +00:00
|
|
|
StorageXDBC::StorageXDBC(
|
2019-12-04 16:06:55 +00:00
|
|
|
const StorageID & table_id_,
|
2018-10-03 10:44:43 +00:00
|
|
|
const std::string & remote_database_name_,
|
|
|
|
const std::string & remote_table_name_,
|
2022-02-12 10:50:05 +00:00
|
|
|
ColumnsDescription columns_,
|
|
|
|
ConstraintsDescription constraints_,
|
2021-04-23 12:18:23 +00:00
|
|
|
const String & comment,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr context_,
|
2018-10-03 10:44:43 +00:00
|
|
|
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 */)
|
2018-10-03 10:44:43 +00:00
|
|
|
, bridge_helper(bridge_helper_)
|
|
|
|
, remote_database_name(remote_database_name_)
|
|
|
|
, remote_table_name(remote_table_name_)
|
2021-09-28 22:17:26 +00:00
|
|
|
, log(&Poco::Logger::get("Storage" + bridge_helper->getName()))
|
2018-10-03 10:44:43 +00:00
|
|
|
{
|
2021-11-01 09:52:27 +00:00
|
|
|
uri = bridge_helper->getMainURI().toString();
|
2018-10-03 10:44:43 +00:00
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2018-10-03 10:44:43 +00:00
|
|
|
std::string StorageXDBC::getReadMethod() const
|
|
|
|
{
|
|
|
|
return Poco::Net::HTTPRequest::HTTP_POST;
|
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2020-06-17 16:39:58 +00:00
|
|
|
std::vector<std::pair<std::string, std::string>> StorageXDBC::getReadURIParams(
|
2021-04-17 08:09:22 +00:00
|
|
|
const Names & /* column_names */,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & /*storage_snapshot*/,
|
2018-10-03 10:44:43 +00:00
|
|
|
const SelectQueryInfo & /*query_info*/,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr /*context*/,
|
2018-10-03 10:44:43 +00:00
|
|
|
QueryProcessingStage::Enum & /*processed_stage*/,
|
2019-02-19 00:48:52 +00:00
|
|
|
size_t max_block_size) const
|
2018-10-03 10:44:43 +00:00
|
|
|
{
|
2021-04-17 08:09:22 +00:00
|
|
|
return bridge_helper->getURLParams(max_block_size);
|
2018-10-03 10:44:43 +00:00
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2020-06-17 16:39:58 +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,
|
2018-10-03 10:44:43 +00:00
|
|
|
const SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr local_context,
|
2018-10-03 10:44:43 +00:00
|
|
|
QueryProcessingStage::Enum & /*processed_stage*/,
|
|
|
|
size_t /*max_block_size*/) const
|
|
|
|
{
|
2023-03-07 20:39:26 +00:00
|
|
|
String query = transformQueryForExternalDatabase(
|
|
|
|
query_info,
|
|
|
|
column_names,
|
2022-02-28 13:29:05 +00:00
|
|
|
columns_description.getOrdinary(),
|
2018-10-03 10:44:43 +00:00
|
|
|
bridge_helper->getIdentifierQuotingStyle(),
|
|
|
|
remote_database_name,
|
|
|
|
remote_table_name,
|
2021-04-10 23:33:54 +00:00
|
|
|
local_context);
|
2021-09-28 22:17:26 +00:00
|
|
|
LOG_TRACE(log, "Query: {}", query);
|
2018-10-03 10:44:43 +00:00
|
|
|
|
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-10-03 10:44:43 +00:00
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2020-08-03 13:54:14 +00:00
|
|
|
Pipe StorageXDBC::read(
|
2020-06-15 19:08:58 +00:00
|
|
|
const Names & column_names,
|
2021-07-09 03:15:41 +00:00
|
|
|
const StorageSnapshotPtr & storage_snapshot,
|
2020-09-20 17:52:17 +00:00
|
|
|
SelectQueryInfo & query_info,
|
2021-04-10 23:33:54 +00:00
|
|
|
ContextPtr local_context,
|
2018-10-03 10:44:43 +00:00
|
|
|
QueryProcessingStage::Enum processed_stage,
|
2019-02-18 23:38:44 +00:00
|
|
|
size_t max_block_size,
|
2022-10-07 10:46:45 +00:00
|
|
|
size_t num_streams)
|
2018-10-03 10:44:43 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
storage_snapshot->check(column_names);
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2018-10-03 10:44:43 +00:00
|
|
|
bridge_helper->startBridgeSync();
|
2021-07-09 03:15:41 +00:00
|
|
|
return IStorageURLBase::read(column_names, storage_snapshot, query_info, local_context, processed_stage, max_block_size, num_streams);
|
2018-10-03 10:44:43 +00:00
|
|
|
}
|
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
|
|
|
|
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(),
|
2020-06-16 14:25:08 +00:00
|
|
|
format_name,
|
2021-04-10 23:33:54 +00:00
|
|
|
getFormatSettings(local_context),
|
2020-06-16 14:25:08 +00:00
|
|
|
metadata_snapshot->getSampleBlock(),
|
2021-04-10 23:33:54 +00:00
|
|
|
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}),
|
2022-08-25 11:18:04 +00:00
|
|
|
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
|
2018-10-03 10:44:43 +00:00
|
|
|
{
|
2022-02-28 13:29:05 +00:00
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
2021-07-09 03:15:41 +00:00
|
|
|
Block StorageXDBC::getHeaderBlock(const Names & column_names, const StorageSnapshotPtr & storage_snapshot) const
|
2018-10-03 10:44:43 +00:00
|
|
|
{
|
2021-07-09 03:15:41 +00:00
|
|
|
return storage_snapshot->getSampleBlockForColumns(column_names);
|
2018-10-03 10:44:43 +00:00
|
|
|
}
|
2018-09-28 02:46:33 +00:00
|
|
|
|
2018-10-03 10:44:43 +00:00
|
|
|
std::string StorageXDBC::getName() const
|
|
|
|
{
|
|
|
|
return bridge_helper->getName();
|
|
|
|
}
|
2018-09-28 02:46:33 +00:00
|
|
|
|
2018-10-03 10:44:43 +00:00
|
|
|
namespace
|
|
|
|
{
|
|
|
|
template <typename BridgeHelperMixin>
|
|
|
|
void registerXDBCStorage(StorageFactory & factory, const std::string & name)
|
2018-09-28 02:46:33 +00:00
|
|
|
{
|
2019-11-11 08:40:00 +00:00
|
|
|
factory.registerStorage(name, [name](const StorageFactory::Arguments & args)
|
2018-10-10 08:38:54 +00:00
|
|
|
{
|
2018-10-03 10:44:43 +00:00
|
|
|
ASTs & engine_args = args.engine_args;
|
|
|
|
|
|
|
|
if (engine_args.size() != 3)
|
2023-01-23 21:13:58 +00:00
|
|
|
throw Exception(ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
|
|
|
|
"Storage {} requires exactly 3 parameters: {}('DSN', database or schema, table)", name, name);
|
2018-10-03 10:44:43 +00:00
|
|
|
|
|
|
|
for (size_t i = 0; i < 3; ++i)
|
2021-04-10 23:33:54 +00:00
|
|
|
engine_args[i] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[i], args.getLocalContext());
|
2018-10-03 10:44:43 +00:00
|
|
|
|
2021-04-10 23:33:54 +00:00
|
|
|
BridgeHelperPtr bridge_helper = std::make_shared<XDBCBridgeHelper<BridgeHelperMixin>>(args.getContext(),
|
|
|
|
args.getContext()->getSettingsRef().http_receive_timeout.value,
|
2023-01-30 16:05:35 +00:00
|
|
|
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,
|
2022-06-23 20:04:06 +00:00
|
|
|
checkAndGetLiteralArgument<String>(engine_args[1], "database_name"),
|
|
|
|
checkAndGetLiteralArgument<String>(engine_args[2], "table_name"),
|
2018-10-03 10:44:43 +00:00
|
|
|
args.columns,
|
2022-02-12 10:50:05 +00:00
|
|
|
args.constraints,
|
2021-04-23 12:18:23 +00:00
|
|
|
args.comment,
|
2021-04-10 23:33:54 +00:00
|
|
|
args.getContext(),
|
2018-10-03 10:44:43 +00:00
|
|
|
bridge_helper);
|
|
|
|
|
2020-04-06 05:19:40 +00:00
|
|
|
},
|
|
|
|
{
|
|
|
|
.source_access_type = BridgeHelperMixin::getSourceAccessType(),
|
2018-10-03 10:44:43 +00:00
|
|
|
});
|
2018-09-28 02:46:33 +00:00
|
|
|
}
|
2018-10-03 10:44:43 +00:00
|
|
|
}
|
2018-09-28 02:46:33 +00:00
|
|
|
|
2018-10-03 10:44:43 +00:00
|
|
|
void registerStorageJDBC(StorageFactory & factory)
|
|
|
|
{
|
|
|
|
registerXDBCStorage<JDBCBridgeMixin>(factory, "JDBC");
|
|
|
|
}
|
2018-09-25 14:29:44 +00:00
|
|
|
|
2018-10-03 10:44:43 +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
|
|
|
}
|