mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-22 15:42:02 +00:00
Added ODBC external dictionary source [#METR-19470].
This commit is contained in:
parent
3688c2270f
commit
348594ffbe
@ -264,6 +264,8 @@ add_library (dbms
|
||||
include/DB/Dictionaries/MySQLDictionarySource.h
|
||||
include/DB/Dictionaries/MongoDBBlockInputStream.h
|
||||
include/DB/Dictionaries/MongoDBDictionarySource.h
|
||||
include/DB/Dictionaries/ODBCBlockInputStream.h
|
||||
include/DB/Dictionaries/ODBCDictionarySource.h
|
||||
include/DB/Dictionaries/TypeCheckingBlockInputStream.h
|
||||
include/DB/Dictionaries/HashedDictionary.h
|
||||
include/DB/Dictionaries/FlatDictionary.h
|
||||
@ -892,6 +894,7 @@ target_link_libraries(dbms
|
||||
librt.a
|
||||
dl
|
||||
mongoclient
|
||||
PocoData
|
||||
libboost_regex.a)
|
||||
|
||||
add_dependencies (dbms
|
||||
|
@ -2,6 +2,7 @@
|
||||
|
||||
#include <DB/Dictionaries/IDictionarySource.h>
|
||||
#include <DB/Dictionaries/DictionaryStructure.h>
|
||||
#include <DB/Dictionaries/ExternalQueryBuilder.h>
|
||||
#include <DB/Client/ConnectionPool.h>
|
||||
#include <DB/DataStreams/RemoteBlockInputStream.h>
|
||||
#include <DB/Interpreters/executeQuery.h>
|
||||
@ -29,11 +30,11 @@ const auto max_connections = 16;
|
||||
class ClickHouseDictionarySource final : public IDictionarySource
|
||||
{
|
||||
public:
|
||||
ClickHouseDictionarySource(const DictionaryStructure & dict_struct,
|
||||
ClickHouseDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix,
|
||||
const Block & sample_block, Context & context)
|
||||
: dict_struct{dict_struct},
|
||||
: dict_struct{dict_struct_},
|
||||
host{config.getString(config_prefix + ".host")},
|
||||
port(config.getInt(config_prefix + ".port")),
|
||||
user{config.getString(config_prefix + ".user", "")},
|
||||
@ -41,14 +42,14 @@ public:
|
||||
db{config.getString(config_prefix + ".db", "")},
|
||||
table{config.getString(config_prefix + ".table")},
|
||||
where{config.getString(config_prefix + ".where", "")},
|
||||
query_builder{dict_struct, db, table, where},
|
||||
sample_block{sample_block}, context(context),
|
||||
is_local{isLocalAddress({ host, port })},
|
||||
pool{is_local ? nullptr : std::make_unique<ConnectionPool>(
|
||||
max_connections, host, port, db, user, password,
|
||||
"ClickHouseDictionarySource")
|
||||
},
|
||||
load_all_query{composeLoadAllQuery()},
|
||||
key_tuple_definition{dict_struct.key ? composeKeyTupleDefinition() : std::string{}}
|
||||
load_all_query{query_builder.composeLoadAllQuery()}
|
||||
{}
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
@ -57,6 +58,7 @@ public:
|
||||
host{other.host}, port{other.port}, user{other.user}, password{other.password},
|
||||
db{other.db}, table{other.table},
|
||||
where{other.where},
|
||||
query_builder{dict_struct, db, table, where},
|
||||
sample_block{other.sample_block}, context(other.context),
|
||||
is_local{other.is_local},
|
||||
pool{is_local ? nullptr : std::make_unique<ConnectionPool>(
|
||||
@ -77,13 +79,16 @@ public:
|
||||
|
||||
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
|
||||
{
|
||||
return createStreamForSelectiveLoad(composeLoadIdsQuery(ids));
|
||||
return createStreamForSelectiveLoad(
|
||||
query_builder.composeLoadIdsQuery(ids));
|
||||
}
|
||||
|
||||
BlockInputStreamPtr loadKeys(
|
||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
|
||||
{
|
||||
return createStreamForSelectiveLoad(composeLoadKeysQuery(key_columns, requested_rows));
|
||||
return createStreamForSelectiveLoad(
|
||||
query_builder.composeLoadKeysQuery(
|
||||
key_columns, requested_rows, ExternalQueryBuilder::IN_WITH_TUPLES));
|
||||
}
|
||||
|
||||
bool isModified() const override { return true; }
|
||||
@ -97,270 +102,6 @@ public:
|
||||
}
|
||||
|
||||
private:
|
||||
std::string composeLoadAllQuery() const
|
||||
{
|
||||
std::string query;
|
||||
|
||||
{
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
if (dict_struct.id)
|
||||
{
|
||||
if (!dict_struct.id.value().expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.value().expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id.value().name, out);
|
||||
|
||||
if (dict_struct.range_min && dict_struct.range_max)
|
||||
{
|
||||
writeString(", ", out);
|
||||
|
||||
if (!dict_struct.range_min.value().expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_min.value().expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_min.value().name, out);
|
||||
|
||||
writeString(", ", out);
|
||||
|
||||
if (!dict_struct.range_max.value().expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.range_max.value().expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.range_max.value().name, out);
|
||||
}
|
||||
}
|
||||
else if (dict_struct.key)
|
||||
{
|
||||
auto first = true;
|
||||
for (const auto & key : *dict_struct.key)
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
|
||||
if (!key.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(key.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(key.name, out);
|
||||
}
|
||||
}
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
{
|
||||
writeString(", ", out);
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(attr.name, out);
|
||||
}
|
||||
|
||||
writeString(" FROM ", out);
|
||||
if (!db.empty())
|
||||
{
|
||||
writeProbablyBackQuotedString(db, out);
|
||||
writeChar('.', out);
|
||||
}
|
||||
writeProbablyBackQuotedString(table, out);
|
||||
|
||||
if (!where.empty())
|
||||
{
|
||||
writeString(" WHERE ", out);
|
||||
writeString(where, out);
|
||||
}
|
||||
|
||||
writeChar(';', out);
|
||||
}
|
||||
|
||||
return query;
|
||||
}
|
||||
|
||||
std::string composeLoadIdsQuery(const std::vector<std::uint64_t> ids)
|
||||
{
|
||||
if (!dict_struct.id)
|
||||
throw Exception{"Simple key required for method", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
std::string query;
|
||||
|
||||
{
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
if (!dict_struct.id.value().expression.empty())
|
||||
{
|
||||
writeParenthesisedString(dict_struct.id.value().expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id.value().name, out);
|
||||
|
||||
for (const auto & attr : dict_struct.attributes)
|
||||
{
|
||||
writeString(", ", out);
|
||||
|
||||
if (!attr.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(attr.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(attr.name, out);
|
||||
}
|
||||
|
||||
writeString(" FROM ", out);
|
||||
if (!db.empty())
|
||||
{
|
||||
writeProbablyBackQuotedString(db, out);
|
||||
writeChar('.', out);
|
||||
}
|
||||
writeProbablyBackQuotedString(table, out);
|
||||
|
||||
writeString(" WHERE ", out);
|
||||
|
||||
if (!where.empty())
|
||||
{
|
||||
writeString(where, out);
|
||||
writeString(" AND ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(dict_struct.id.value().name, out);
|
||||
writeString(" IN (", out);
|
||||
|
||||
auto first = true;
|
||||
for (const auto id : ids)
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
writeString(DB::toString(id), out);
|
||||
}
|
||||
|
||||
writeString(");", out);
|
||||
}
|
||||
|
||||
return query;
|
||||
}
|
||||
|
||||
std::string composeLoadKeysQuery(
|
||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||
{
|
||||
if (!dict_struct.key)
|
||||
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
std::string query;
|
||||
|
||||
{
|
||||
WriteBufferFromString out{query};
|
||||
writeString("SELECT ", out);
|
||||
|
||||
auto first = true;
|
||||
for (const auto & key_or_attribute : boost::join(*dict_struct.key, dict_struct.attributes))
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
|
||||
if (!key_or_attribute.expression.empty())
|
||||
{
|
||||
writeParenthesisedString(key_or_attribute.expression, out);
|
||||
writeString(" AS ", out);
|
||||
}
|
||||
|
||||
writeProbablyBackQuotedString(key_or_attribute.name, out);
|
||||
}
|
||||
|
||||
writeString(" FROM ", out);
|
||||
if (!db.empty())
|
||||
{
|
||||
writeProbablyBackQuotedString(db, out);
|
||||
writeChar('.', out);
|
||||
}
|
||||
writeProbablyBackQuotedString(table, out);
|
||||
|
||||
writeString(" WHERE ", out);
|
||||
|
||||
if (!where.empty())
|
||||
{
|
||||
writeString(where, out);
|
||||
writeString(" AND ", out);
|
||||
}
|
||||
|
||||
writeString(key_tuple_definition, out);
|
||||
writeString(" IN (", out);
|
||||
|
||||
first = true;
|
||||
for (const auto row : requested_rows)
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
composeKeyTuple(key_columns, row, out);
|
||||
}
|
||||
|
||||
writeString(");", out);
|
||||
}
|
||||
|
||||
return query;
|
||||
}
|
||||
|
||||
std::string composeKeyTupleDefinition() const
|
||||
{
|
||||
if (!dict_struct.key)
|
||||
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
std::string result{"("};
|
||||
|
||||
auto first = true;
|
||||
for (const auto & key : *dict_struct.key)
|
||||
{
|
||||
if (!first)
|
||||
result += ", ";
|
||||
|
||||
first = false;
|
||||
result += key.name;
|
||||
}
|
||||
|
||||
result += ")";
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
void composeKeyTuple(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
|
||||
{
|
||||
writeString("(", out);
|
||||
|
||||
const auto keys_size = key_columns.size();
|
||||
auto first = true;
|
||||
for (const auto i : ext::range(0, keys_size))
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
(*dict_struct.key)[i].type->serializeTextQuoted(*key_columns[i], row, out);
|
||||
}
|
||||
|
||||
writeString(")", out);
|
||||
}
|
||||
|
||||
BlockInputStreamPtr createStreamForSelectiveLoad(const std::string query)
|
||||
{
|
||||
@ -369,6 +110,7 @@ private:
|
||||
return new RemoteBlockInputStream{pool.get(), query, nullptr};
|
||||
}
|
||||
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const std::string host;
|
||||
const UInt16 port;
|
||||
@ -377,12 +119,12 @@ private:
|
||||
const std::string db;
|
||||
const std::string table;
|
||||
const std::string where;
|
||||
ExternalQueryBuilder query_builder;
|
||||
Block sample_block;
|
||||
Context & context;
|
||||
const bool is_local;
|
||||
std::unique_ptr<ConnectionPool> pool;
|
||||
const std::string load_all_query;
|
||||
const std::string key_tuple_definition;
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -6,6 +6,7 @@
|
||||
#include <DB/Dictionaries/MySQLDictionarySource.h>
|
||||
#include <DB/Dictionaries/ClickHouseDictionarySource.h>
|
||||
#include <DB/Dictionaries/MongoDBDictionarySource.h>
|
||||
#include <DB/Dictionaries/ODBCDictionarySource.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <common/singleton.h>
|
||||
#include <memory>
|
||||
@ -106,14 +107,16 @@ public:
|
||||
}
|
||||
else if ("mongodb" == source_type)
|
||||
{
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb",
|
||||
sample_block, context);
|
||||
return std::make_unique<MongoDBDictionarySource>(dict_struct, config, config_prefix + ".mongodb", sample_block);
|
||||
}
|
||||
else if ("odbc" == source_type)
|
||||
{
|
||||
return std::make_unique<ODBCDictionarySource>(dict_struct, config, config_prefix + ".odbc", sample_block);
|
||||
}
|
||||
|
||||
throw Exception{
|
||||
name + ": unknown dictionary source type: " + source_type,
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG
|
||||
};
|
||||
ErrorCodes::UNKNOWN_ELEMENT_IN_CONFIG};
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -198,9 +198,22 @@ struct ExternalQueryBuilder
|
||||
return query;
|
||||
}
|
||||
|
||||
/** Получить запрос на загрузку данных по множеству сложных ключей. */
|
||||
|
||||
/** Получить запрос на загрузку данных по множеству сложных ключей.
|
||||
* Есть два метода их указания в секции WHERE:
|
||||
* 1. (x = c11 AND y = c12) OR (x = c21 AND y = c22) ...
|
||||
* 2. (x, y) IN ((c11, c12), (c21, c22), ...)
|
||||
*/
|
||||
enum LoadKeysMethod
|
||||
{
|
||||
AND_OR_CHAIN,
|
||||
IN_WITH_TUPLES,
|
||||
};
|
||||
|
||||
std::string composeLoadKeysQuery(
|
||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows)
|
||||
const ConstColumnPlainPtrs & key_columns,
|
||||
const std::vector<std::size_t> & requested_rows,
|
||||
LoadKeysMethod method)
|
||||
{
|
||||
if (!dict_struct.key)
|
||||
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
@ -240,18 +253,44 @@ struct ExternalQueryBuilder
|
||||
|
||||
if (!where.empty())
|
||||
{
|
||||
writeString("(", out);
|
||||
writeString(where, out);
|
||||
writeString(" AND ", out);
|
||||
writeString(") AND (", out);
|
||||
}
|
||||
|
||||
first = true;
|
||||
for (const auto row : requested_rows)
|
||||
if (method == AND_OR_CHAIN)
|
||||
{
|
||||
if (!first)
|
||||
writeString(" OR ", out);
|
||||
first = true;
|
||||
for (const auto row : requested_rows)
|
||||
{
|
||||
if (!first)
|
||||
writeString(" OR ", out);
|
||||
|
||||
first = false;
|
||||
composeKeyCondition(key_columns, row, out);
|
||||
first = false;
|
||||
composeKeyCondition(key_columns, row, out);
|
||||
}
|
||||
}
|
||||
else if (method == IN_WITH_TUPLES)
|
||||
{
|
||||
writeString(composeKeyTupleDefinition(), out);
|
||||
writeString(" IN (", out);
|
||||
|
||||
first = true;
|
||||
for (const auto row : requested_rows)
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
composeKeyTuple(key_columns, row, out);
|
||||
}
|
||||
|
||||
writeString(")", out);
|
||||
}
|
||||
|
||||
if (!where.empty())
|
||||
{
|
||||
writeString(")", out);
|
||||
}
|
||||
|
||||
writeString(";", out);
|
||||
@ -260,7 +299,9 @@ struct ExternalQueryBuilder
|
||||
return query;
|
||||
}
|
||||
|
||||
|
||||
private:
|
||||
/// Выражение вида (x = c1 AND y = c2 ...)
|
||||
void composeKeyCondition(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
|
||||
{
|
||||
writeString("(", out);
|
||||
@ -284,6 +325,48 @@ private:
|
||||
|
||||
writeString(")", out);
|
||||
}
|
||||
|
||||
/// Выражение вида (x, y, ...)
|
||||
std::string composeKeyTupleDefinition() const
|
||||
{
|
||||
if (!dict_struct.key)
|
||||
throw Exception{"Composite key required for method", ErrorCodes::UNSUPPORTED_METHOD};
|
||||
|
||||
std::string result{"("};
|
||||
|
||||
auto first = true;
|
||||
for (const auto & key : *dict_struct.key)
|
||||
{
|
||||
if (!first)
|
||||
result += ", ";
|
||||
|
||||
first = false;
|
||||
result += key.name;
|
||||
}
|
||||
|
||||
result += ")";
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
/// Выражение вида (c1, c2, ...)
|
||||
void composeKeyTuple(const ConstColumnPlainPtrs & key_columns, const std::size_t row, WriteBuffer & out) const
|
||||
{
|
||||
writeString("(", out);
|
||||
|
||||
const auto keys_size = key_columns.size();
|
||||
auto first = true;
|
||||
for (const auto i : ext::range(0, keys_size))
|
||||
{
|
||||
if (!first)
|
||||
writeString(", ", out);
|
||||
|
||||
first = false;
|
||||
(*dict_struct.key)[i].type->serializeTextQuoted(*key_columns[i], row, out);
|
||||
}
|
||||
|
||||
writeString(")", out);
|
||||
}
|
||||
};
|
||||
|
||||
}
|
||||
|
@ -29,9 +29,9 @@ class MongoDBDictionarySource final : public IDictionarySource
|
||||
const DictionaryStructure & dict_struct, const std::string & host, const std::string & port,
|
||||
const std::string & user, const std::string & password,
|
||||
const std::string & db, const std::string & collection,
|
||||
const Block & sample_block, Context & context)
|
||||
const Block & sample_block)
|
||||
: dict_struct{dict_struct}, host{host}, port{port}, user{user}, password{password},
|
||||
db{db}, collection{collection}, sample_block{sample_block}, context(context),
|
||||
db{db}, collection{collection}, sample_block{sample_block},
|
||||
connection{true}
|
||||
{
|
||||
init();
|
||||
@ -76,7 +76,7 @@ class MongoDBDictionarySource final : public IDictionarySource
|
||||
public:
|
||||
MongoDBDictionarySource(
|
||||
const DictionaryStructure & dict_struct, const Poco::Util::AbstractConfiguration & config,
|
||||
const std::string & config_prefix, Block & sample_block, Context & context)
|
||||
const std::string & config_prefix, Block & sample_block)
|
||||
: MongoDBDictionarySource{
|
||||
dict_struct,
|
||||
config.getString(config_prefix + ".host"),
|
||||
@ -85,7 +85,7 @@ public:
|
||||
config.getString(config_prefix + ".password", ""),
|
||||
config.getString(config_prefix + ".db", ""),
|
||||
config.getString(config_prefix + ".collection"),
|
||||
sample_block, context
|
||||
sample_block
|
||||
}
|
||||
{
|
||||
}
|
||||
@ -93,8 +93,7 @@ public:
|
||||
MongoDBDictionarySource(const MongoDBDictionarySource & other)
|
||||
: MongoDBDictionarySource{
|
||||
other.dict_struct, other.host, other.port, other.user, other.password,
|
||||
other.db, other.collection, other.sample_block, other.context
|
||||
}
|
||||
other.db, other.collection, other.sample_block}
|
||||
{
|
||||
}
|
||||
|
||||
@ -188,7 +187,6 @@ private:
|
||||
const std::string db;
|
||||
const std::string collection;
|
||||
Block sample_block;
|
||||
Context & context;
|
||||
|
||||
mongo::DBClientConnection connection;
|
||||
mongo::BSONObj fields_to_query;
|
||||
|
@ -66,7 +66,7 @@ public:
|
||||
{
|
||||
/// Здесь не логгируем и не обновляем время модификации, так как запрос может быть большим, и часто задаваться.
|
||||
|
||||
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows);
|
||||
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
|
||||
return new MySQLBlockInputStream{pool.Get(), query, sample_block, max_block_size};
|
||||
}
|
||||
|
||||
|
135
dbms/include/DB/Dictionaries/ODBCBlockInputStream.h
Normal file
135
dbms/include/DB/Dictionaries/ODBCBlockInputStream.h
Normal file
@ -0,0 +1,135 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Core/Block.h>
|
||||
#include <DB/DataStreams/IProfilingBlockInputStream.h>
|
||||
#include <DB/DataTypes/DataTypesNumberFixed.h>
|
||||
#include <DB/DataTypes/DataTypeString.h>
|
||||
#include <DB/DataTypes/DataTypeDate.h>
|
||||
#include <DB/DataTypes/DataTypeDateTime.h>
|
||||
#include <DB/Columns/ColumnString.h>
|
||||
#include <DB/Dictionaries/ExternalResultDescription.h>
|
||||
|
||||
#include <Poco/Data/Session.h>
|
||||
#include <Poco/Data/Statement.h>
|
||||
#include <Poco/Data/RecordSet.h>
|
||||
|
||||
#include <ext/range.hpp>
|
||||
#include <vector>
|
||||
#include <string>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
namespace ErrorCodes
|
||||
{
|
||||
extern const int NUMBER_OF_COLUMNS_DOESNT_MATCH;
|
||||
}
|
||||
|
||||
|
||||
/// Allows processing results of a query to ODBC source as a sequence of Blocks, simplifies chaining
|
||||
class ODBCBlockInputStream final : public IProfilingBlockInputStream
|
||||
{
|
||||
public:
|
||||
ODBCBlockInputStream(
|
||||
Poco::Data::Session && session, const std::string & query_str, const Block & sample_block,
|
||||
const std::size_t max_block_size)
|
||||
:
|
||||
session{session},
|
||||
statement{this->session << query_str},
|
||||
result{statement},
|
||||
iterator{result.begin()},
|
||||
max_block_size{max_block_size}
|
||||
{
|
||||
if (sample_block.columns() != result.columnCount())
|
||||
throw Exception{
|
||||
"RecordSet contains " + toString(result.columnCount()) + " columns while " +
|
||||
toString(sample_block.columns()) + " expected",
|
||||
ErrorCodes::NUMBER_OF_COLUMNS_DOESNT_MATCH};
|
||||
|
||||
description.init(sample_block);
|
||||
}
|
||||
|
||||
String getName() const override { return "ODBC"; }
|
||||
|
||||
String getID() const override
|
||||
{
|
||||
return "ODBC(" + statement.toString() + ")";
|
||||
}
|
||||
|
||||
private:
|
||||
using value_type_t = ExternalResultDescription::value_type_t;
|
||||
|
||||
|
||||
Block readImpl() override
|
||||
{
|
||||
if (iterator == result.end())
|
||||
return {};
|
||||
|
||||
auto block = description.sample_block.cloneEmpty();
|
||||
|
||||
/// cache pointers returned by the calls to getByPosition
|
||||
std::vector<IColumn *> columns(block.columns());
|
||||
for (const auto i : ext::range(0, columns.size()))
|
||||
columns[i] = block.getByPosition(i).column.get();
|
||||
|
||||
std::size_t num_rows = 0;
|
||||
while (iterator != result.end())
|
||||
{
|
||||
Poco::Data::Row & row = *iterator;
|
||||
|
||||
for (const auto idx : ext::range(0, row.fieldCount()))
|
||||
{
|
||||
const Poco::Dynamic::Var & value = row[idx];
|
||||
|
||||
if (!value.isEmpty())
|
||||
insertValue(columns[idx], description.types[idx], value);
|
||||
else
|
||||
insertDefaultValue(columns[idx], *description.sample_columns[idx]);
|
||||
}
|
||||
|
||||
++num_rows;
|
||||
if (num_rows == max_block_size)
|
||||
break;
|
||||
|
||||
++iterator;
|
||||
}
|
||||
|
||||
return block;
|
||||
}
|
||||
|
||||
static void insertValue(IColumn * const column, const value_type_t type, const Poco::Dynamic::Var & value)
|
||||
{
|
||||
switch (type)
|
||||
{
|
||||
case value_type_t::UInt8: static_cast<ColumnUInt8 *>(column)->insert(value.convert<UInt64>()); break;
|
||||
case value_type_t::UInt16: static_cast<ColumnUInt16 *>(column)->insert(value.convert<UInt64>()); break;
|
||||
case value_type_t::UInt32: static_cast<ColumnUInt32 *>(column)->insert(value.convert<UInt64>()); break;
|
||||
case value_type_t::UInt64: static_cast<ColumnUInt64 *>(column)->insert(value.convert<UInt64>()); break;
|
||||
case value_type_t::Int8: static_cast<ColumnInt8 *>(column)->insert(value.convert<Int64>()); break;
|
||||
case value_type_t::Int16: static_cast<ColumnInt16 *>(column)->insert(value.convert<Int64>()); break;
|
||||
case value_type_t::Int32: static_cast<ColumnInt32 *>(column)->insert(value.convert<Int64>()); break;
|
||||
case value_type_t::Int64: static_cast<ColumnInt64 *>(column)->insert(value.convert<Int64>()); break;
|
||||
case value_type_t::Float32: static_cast<ColumnFloat32 *>(column)->insert(value.convert<Float64>()); break;
|
||||
case value_type_t::Float64: static_cast<ColumnFloat64 *>(column)->insert(value.convert<Float64>()); break;
|
||||
case value_type_t::String: static_cast<ColumnString *>(column)->insert(value.convert<String>()); break;
|
||||
case value_type_t::Date: static_cast<ColumnUInt16 *>(column)->insert(UInt16{LocalDate{value.convert<String>()}.getDayNum()}); break;
|
||||
case value_type_t::DateTime: static_cast<ColumnUInt32 *>(column)->insert(time_t{LocalDateTime{value.convert<String>()}}); break;
|
||||
}
|
||||
}
|
||||
|
||||
static void insertDefaultValue(IColumn * const column, const IColumn & sample_column)
|
||||
{
|
||||
column->insertFrom(sample_column, 0);
|
||||
}
|
||||
|
||||
Poco::Data::Session session;
|
||||
Poco::Data::Statement statement;
|
||||
Poco::Data::RecordSet result;
|
||||
Poco::Data::RecordSet::Iterator iterator;
|
||||
|
||||
const std::size_t max_block_size;
|
||||
ExternalResultDescription description;
|
||||
};
|
||||
|
||||
}
|
96
dbms/include/DB/Dictionaries/ODBCDictionarySource.h
Normal file
96
dbms/include/DB/Dictionaries/ODBCDictionarySource.h
Normal file
@ -0,0 +1,96 @@
|
||||
#pragma once
|
||||
|
||||
#include <DB/Dictionaries/IDictionarySource.h>
|
||||
#include <DB/Dictionaries/ODBCBlockInputStream.h>
|
||||
#include <DB/Dictionaries/ExternalQueryBuilder.h>
|
||||
#include <ext/range.hpp>
|
||||
#include <mysqlxx/Pool.h>
|
||||
#include <Poco/Util/AbstractConfiguration.h>
|
||||
#include <Poco/Data/SessionPool.h>
|
||||
|
||||
|
||||
namespace DB
|
||||
{
|
||||
|
||||
|
||||
/// Allows loading dictionaries from a ODBC source
|
||||
class ODBCDictionarySource final : public IDictionarySource
|
||||
{
|
||||
static const auto max_block_size = 8192;
|
||||
|
||||
public:
|
||||
ODBCDictionarySource(const DictionaryStructure & dict_struct_,
|
||||
const Poco::Util::AbstractConfiguration & config, const std::string & config_prefix,
|
||||
const Block & sample_block)
|
||||
: dict_struct{dict_struct_},
|
||||
db{config.getString(config_prefix + ".db", "")},
|
||||
table{config.getString(config_prefix + ".table")},
|
||||
where{config.getString(config_prefix + ".where", "")},
|
||||
sample_block{sample_block},
|
||||
pool{std::make_shared<Poco::Data::SessionPool>(
|
||||
config.getString(config_prefix + ".connector", "ODBC"),
|
||||
config.getString(config_prefix + ".connection_string"))},
|
||||
query_builder{dict_struct, db, table, where},
|
||||
load_all_query{query_builder.composeLoadAllQuery()}
|
||||
{}
|
||||
|
||||
/// copy-constructor is provided in order to support cloneability
|
||||
ODBCDictionarySource(const ODBCDictionarySource & other)
|
||||
: dict_struct{other.dict_struct},
|
||||
db{other.db},
|
||||
table{other.table},
|
||||
where{other.where},
|
||||
sample_block{other.sample_block},
|
||||
pool{other.pool},
|
||||
query_builder{dict_struct, db, table, where},
|
||||
load_all_query{other.load_all_query}
|
||||
{}
|
||||
|
||||
BlockInputStreamPtr loadAll() override
|
||||
{
|
||||
LOG_TRACE(log, load_all_query);
|
||||
return new ODBCBlockInputStream{pool->get(), load_all_query, sample_block, max_block_size};
|
||||
}
|
||||
|
||||
BlockInputStreamPtr loadIds(const std::vector<std::uint64_t> & ids) override
|
||||
{
|
||||
const auto query = query_builder.composeLoadIdsQuery(ids);
|
||||
return new ODBCBlockInputStream{pool->get(), query, sample_block, max_block_size};
|
||||
}
|
||||
|
||||
BlockInputStreamPtr loadKeys(
|
||||
const ConstColumnPlainPtrs & key_columns, const std::vector<std::size_t> & requested_rows) override
|
||||
{
|
||||
const auto query = query_builder.composeLoadKeysQuery(key_columns, requested_rows, ExternalQueryBuilder::AND_OR_CHAIN);
|
||||
return new ODBCBlockInputStream{pool->get(), query, sample_block, max_block_size};
|
||||
}
|
||||
|
||||
bool isModified() const override
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
bool supportsSelectiveLoad() const override { return true; }
|
||||
|
||||
DictionarySourcePtr clone() const override { return std::make_unique<ODBCDictionarySource>(*this); }
|
||||
|
||||
std::string toString() const override
|
||||
{
|
||||
return "ODBC: " + db + '.' + table + (where.empty() ? "" : ", where: " + where);
|
||||
}
|
||||
|
||||
private:
|
||||
Logger * log = &Logger::get("ODBCDictionarySource");
|
||||
|
||||
const DictionaryStructure dict_struct;
|
||||
const std::string db;
|
||||
const std::string table;
|
||||
const std::string where;
|
||||
Block sample_block;
|
||||
std::shared_ptr<Poco::Data::SessionPool> pool;
|
||||
ExternalQueryBuilder query_builder;
|
||||
const std::string load_all_query;
|
||||
};
|
||||
|
||||
|
||||
}
|
Loading…
Reference in New Issue
Block a user