mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Better joinGet.
This commit is contained in:
parent
02f236e457
commit
d8eba7e71b
@ -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]);
|
||||
|
@ -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; }
|
||||
|
@ -17,3 +17,5 @@ abc
|
||||
def
|
||||
|
||||
[0] 1
|
||||
0
|
||||
0
|
||||
|
@ -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;
|
||||
|
Loading…
Reference in New Issue
Block a user