2017-04-01 09:19:00 +00:00
|
|
|
#include <TableFunctions/getStructureOfRemoteTable.h>
|
|
|
|
#include <Storages/StorageDistributed.h>
|
|
|
|
#include <Parsers/ASTIdentifier.h>
|
|
|
|
#include <Parsers/ASTLiteral.h>
|
|
|
|
#include <Parsers/ASTFunction.h>
|
|
|
|
#include <Interpreters/evaluateConstantExpression.h>
|
|
|
|
#include <Interpreters/Cluster.h>
|
|
|
|
#include <Interpreters/Context.h>
|
|
|
|
#include <Interpreters/getClusterName.h>
|
|
|
|
#include <Common/SipHash.h>
|
|
|
|
#include <TableFunctions/TableFunctionShardByHash.h>
|
2017-06-10 09:04:31 +00:00
|
|
|
#include <TableFunctions/TableFunctionFactory.h>
|
2016-05-13 03:22:16 +00:00
|
|
|
|
|
|
|
|
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
|
|
|
|
extern const int BAD_ARGUMENTS;
|
2016-05-13 03:22:16 +00:00
|
|
|
}
|
|
|
|
|
2017-05-23 18:46:52 +00:00
|
|
|
StoragePtr TableFunctionShardByHash::execute(const ASTPtr & ast_function, const Context & context) const
|
2016-05-13 03:22:16 +00:00
|
|
|
{
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTs & args_func = typeid_cast<ASTFunction &>(*ast_function).children;
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
const char * err = "Table function 'shardByHash' requires 4 parameters: "
|
|
|
|
"cluster name, key string to hash, name of remote database, name of remote table.";
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (args_func.size() != 1)
|
|
|
|
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
ASTs & args = typeid_cast<ASTExpressionList &>(*args_func.at(0)).children;
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (args.size() != 4)
|
|
|
|
throw Exception(err, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
String cluster_name;
|
|
|
|
String key;
|
|
|
|
String remote_database;
|
|
|
|
String remote_table;
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto getStringLiteral = [](const IAST & node, const char * description)
|
|
|
|
{
|
|
|
|
const ASTLiteral * lit = typeid_cast<const ASTLiteral *>(&node);
|
|
|
|
if (!lit)
|
|
|
|
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
if (lit->value.getType() != Field::Types::String)
|
|
|
|
throw Exception(description + String(" must be string literal (in single quotes)."), ErrorCodes::BAD_ARGUMENTS);
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
return safeGet<const String &>(lit->value);
|
|
|
|
};
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
cluster_name = getClusterName(*args[0]);
|
|
|
|
key = getStringLiteral(*args[1], "Key to hash");
|
2016-08-25 17:23:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
args[2] = evaluateConstantExpressionOrIdentidierAsLiteral(args[2], context);
|
|
|
|
args[3] = evaluateConstantExpressionOrIdentidierAsLiteral(args[3], context);
|
2016-08-25 17:23:29 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
remote_database = static_cast<const ASTLiteral &>(*args[2]).value.safeGet<String>();
|
|
|
|
remote_table = static_cast<const ASTLiteral &>(*args[3]).value.safeGet<String>();
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
/// Similar to other TableFunctions.
|
|
|
|
for (auto & arg : args)
|
|
|
|
if (ASTIdentifier * id = typeid_cast<ASTIdentifier *>(arg.get()))
|
|
|
|
id->kind = ASTIdentifier::Table;
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
auto cluster = context.getCluster(cluster_name);
|
|
|
|
size_t shard_index = sipHash64(key) % cluster->getShardCount();
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-04-01 07:20:54 +00:00
|
|
|
std::shared_ptr<Cluster> shard(cluster->getClusterWithSingleShard(shard_index).release());
|
2016-05-13 03:22:16 +00:00
|
|
|
|
2017-06-06 18:48:38 +00:00
|
|
|
auto res = StorageDistributed::createWithOwnCluster(
|
2017-04-01 07:20:54 +00:00
|
|
|
getName(),
|
|
|
|
std::make_shared<NamesAndTypesList>(getStructureOfRemoteTable(*shard, remote_database, remote_table, context)),
|
|
|
|
remote_database,
|
|
|
|
remote_table,
|
|
|
|
shard,
|
|
|
|
context);
|
2017-06-06 17:06:14 +00:00
|
|
|
res->startup();
|
|
|
|
return res;
|
2016-05-13 03:22:16 +00:00
|
|
|
}
|
|
|
|
|
2017-06-10 09:04:31 +00:00
|
|
|
|
|
|
|
void registerTableFunctionShardByHash(TableFunctionFactory & factory)
|
|
|
|
{
|
|
|
|
TableFunctionFactory::instance().registerFunction<TableFunctionShardByHash>();
|
|
|
|
}
|
|
|
|
|
2016-05-13 03:22:16 +00:00
|
|
|
}
|