Better joinGet.

This commit is contained in:
Amos Bird 2019-01-25 22:50:31 +08:00
parent 02f236e457
commit d8eba7e71b
4 changed files with 71 additions and 16 deletions

View File

@ -1,10 +1,10 @@
#include <Functions/FunctionJoinGet.h>
#include <Functions/FunctionFactory.h>
#include <Functions/FunctionHelpers.h>
#include <Interpreters/Context.h>
#include <Interpreters/Join.h>
#include <Storages/StorageJoin.h>
#include <Functions/FunctionFactory.h>
namespace DB
@ -15,10 +15,10 @@ namespace ErrorCodes
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
}
FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const
static auto getJoin(const ColumnsWithTypeAndName & arguments, const Context & context)
{
if (arguments.size() != 3)
throw Exception{"Function " + getName() + " takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
throw Exception{"Function joinGet takes 3 arguments", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
String join_name;
if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get()))
@ -26,37 +26,62 @@ FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName &
join_name = name_col->getValue<String>();
}
else
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
+ ", expected a const string.",
throw Exception{"Illegal type " + arguments[0].type->getName() + " of first argument of function joinGet, expected a const string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto table = context.getTable("", join_name);
StorageJoin * storage_join = dynamic_cast<StorageJoin *>(table.get());
size_t dot = join_name.find('.');
String database_name;
if (dot == String::npos)
{
database_name = context.getCurrentDatabase();
dot = 0;
}
else
{
database_name = join_name.substr(0, dot);
++dot;
}
String table_name = join_name.substr(dot);
auto table = context.getTable(database_name, table_name);
auto storage_join = std::dynamic_pointer_cast<StorageJoin>(table);
if (!storage_join)
throw Exception{"Table " + join_name + " should have engine StorageJoin", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
auto join = storage_join->getJoin();
String attr_name;
if (auto name_col = checkAndGetColumnConst<ColumnString>(arguments[1].column.get()))
{
attr_name = name_col->getValue<String>();
}
else
throw Exception{"Illegal type " + arguments[1].type->getName() + " of second argument of function " + getName()
+ ", expected a const string.",
throw Exception{"Illegal type " + arguments[1].type->getName()
+ " of second argument of function joinGet, expected a const string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return std::make_pair(storage_join, attr_name);
}
FunctionBasePtr FunctionBuilderJoinGet::buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const
{
auto [storage_join, attr_name] = getJoin(arguments, context);
auto join = storage_join->getJoin();
DataTypes data_types(arguments.size());
auto table_lock = storage_join->lockStructure(false);
for (size_t i = 0; i < arguments.size(); ++i)
data_types[i] = arguments[i].type;
auto return_type = join->joinGetReturnType(attr_name);
return std::make_shared<DefaultFunction>(
std::make_shared<FunctionJoinGet>(join, attr_name), data_types, join->joinGetReturnType(attr_name));
std::make_shared<FunctionJoinGet>(table_lock, storage_join, join, attr_name, return_type), data_types, return_type);
}
DataTypePtr FunctionBuilderJoinGet::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
{
auto [storage_join, attr_name] = getJoin(arguments, context);
auto join = storage_join->getJoin();
return join->joinGetReturnType(attr_name);
}
void FunctionJoinGet::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
{
auto & ctn = block.getByPosition(arguments[2]);

View File

@ -3,20 +3,32 @@
namespace DB
{
class Context;
class IStorage;
using StoragePtr = std::shared_ptr<IStorage>;
class Join;
using JoinPtr = std::shared_ptr<Join>;
class TableStructureReadLock;
using TableStructureReadLockPtr = std::shared_ptr<TableStructureReadLock>;
class FunctionJoinGet final : public IFunction, public std::enable_shared_from_this<FunctionJoinGet>
{
public:
static constexpr auto name = "joinGet";
FunctionJoinGet(JoinPtr join, const String & attr_name) : join(std::move(join)), attr_name(attr_name) {}
FunctionJoinGet(
TableStructureReadLockPtr table_lock, StoragePtr storage_join, JoinPtr join, const String & attr_name, DataTypePtr return_type)
: table_lock(std::move(table_lock))
, storage_join(std::move(storage_join))
, join(std::move(join))
, attr_name(attr_name)
, return_type(std::move(return_type))
{
}
String getName() const override { return name; }
protected:
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; }
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return return_type; }
void executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t input_rows_count) override;
private:
@ -24,8 +36,11 @@ private:
size_t getNumberOfArguments() const override { return 0; }
private:
TableStructureReadLockPtr table_lock;
StoragePtr storage_join;
JoinPtr join;
const String attr_name;
DataTypePtr return_type;
};
class FunctionBuilderJoinGet final : public FunctionBuilderImpl
@ -40,7 +55,7 @@ public:
protected:
FunctionBasePtr buildImpl(const ColumnsWithTypeAndName & arguments, const DataTypePtr &) const override;
DataTypePtr getReturnTypeImpl(const DataTypes & /*arguments*/) const override { return nullptr; }
DataTypePtr getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const override;
private:
bool isVariadic() const override { return true; }

View File

@ -17,3 +17,5 @@ abc
def
[0] 1
0
0

View File

@ -49,3 +49,16 @@ DROP TABLE test.join_any_left_null;
DROP TABLE test.join_all_inner;
DROP TABLE test.join_all_left;
DROP TABLE test.join_string_key;
-- test provided by Alexander Zaitsev
DROP TABLE IF EXISTS test.join_test;
CREATE TABLE test.join_test (a UInt8, b UInt8) Engine = Join(ANY, LEFT, a);
USE test;
select joinGet('join_test', 'b', 1);
USE system;
SELECT joinGet('test.join_test', 'b', 1);
USE default;
DROP TABLE test.join_test;