2017-10-17 12:02:30 +00:00
|
|
|
#include <Functions/FunctionsExternalModels.h>
|
|
|
|
#include <Functions/FunctionHelpers.h>
|
|
|
|
#include <Functions/FunctionFactory.h>
|
|
|
|
|
2017-10-17 10:44:46 +00:00
|
|
|
#include <Interpreters/Context.h>
|
2017-10-17 12:02:30 +00:00
|
|
|
#include <Interpreters/ExternalModels.h>
|
2017-10-17 10:44:46 +00:00
|
|
|
#include <DataTypes/DataTypeString.h>
|
|
|
|
#include <DataTypes/DataTypesNumber.h>
|
|
|
|
#include <Columns/ColumnString.h>
|
|
|
|
#include <ext/range.h>
|
2018-04-24 07:16:39 +00:00
|
|
|
#include <string>
|
|
|
|
#include <memory>
|
2018-12-26 16:44:57 +00:00
|
|
|
#include <DataTypes/DataTypeNullable.h>
|
|
|
|
#include <Columns/ColumnNullable.h>
|
|
|
|
#include <Columns/ColumnTuple.h>
|
|
|
|
#include <DataTypes/DataTypeTuple.h>
|
2017-10-17 10:43:42 +00:00
|
|
|
|
2017-10-17 10:44:46 +00:00
|
|
|
namespace DB
|
|
|
|
{
|
|
|
|
|
|
|
|
FunctionPtr FunctionModelEvaluate::create(const Context & context)
|
|
|
|
{
|
|
|
|
return std::make_shared<FunctionModelEvaluate>(context.getExternalModels());
|
|
|
|
}
|
|
|
|
|
|
|
|
namespace ErrorCodes
|
|
|
|
{
|
|
|
|
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
|
2018-12-07 03:20:27 +00:00
|
|
|
extern const int TOO_FEW_ARGUMENTS_FOR_FUNCTION;
|
2017-10-17 10:44:46 +00:00
|
|
|
extern const int ILLEGAL_COLUMN;
|
|
|
|
}
|
|
|
|
|
2018-12-26 16:44:57 +00:00
|
|
|
DataTypePtr FunctionModelEvaluate::getReturnTypeImpl(const ColumnsWithTypeAndName & arguments) const
|
2017-10-17 10:44:46 +00:00
|
|
|
{
|
|
|
|
if (arguments.size() < 2)
|
|
|
|
throw Exception("Function " + getName() + " expects at least 2 arguments",
|
2018-12-07 03:20:27 +00:00
|
|
|
ErrorCodes::TOO_FEW_ARGUMENTS_FOR_FUNCTION);
|
2017-10-17 10:44:46 +00:00
|
|
|
|
2018-12-26 16:44:57 +00:00
|
|
|
if (!isString(arguments[0].type))
|
|
|
|
throw Exception("Illegal type " + arguments[0].type->getName() + " of first argument of function " + getName()
|
2017-10-17 10:44:46 +00:00
|
|
|
+ ", expected a string.", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
|
|
|
|
2018-12-26 16:44:57 +00:00
|
|
|
const auto name_col = checkAndGetColumnConst<ColumnString>(arguments[0].column.get());
|
|
|
|
if (!name_col)
|
|
|
|
throw Exception("First argument of function " + getName() + " must be a constant string",
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
|
|
|
bool has_nullable = false;
|
|
|
|
for (size_t i = 1; i < arguments.size(); ++i)
|
|
|
|
has_nullable = has_nullable || arguments[i].type->isNullable();
|
|
|
|
|
|
|
|
auto model = models.getModel(name_col->getValue<String>());
|
|
|
|
auto type = model->getReturnType();
|
|
|
|
|
|
|
|
if (has_nullable)
|
|
|
|
{
|
|
|
|
if (auto * tuple = typeid_cast<const DataTypeTuple *>(type.get()))
|
|
|
|
{
|
|
|
|
auto elements = tuple->getElements();
|
|
|
|
for (auto & element : elements)
|
|
|
|
element = makeNullable(element);
|
|
|
|
|
|
|
|
type = std::make_shared<DataTypeTuple>(elements);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
type = makeNullable(type);
|
|
|
|
}
|
|
|
|
|
|
|
|
return type;
|
2017-10-17 10:44:46 +00:00
|
|
|
}
|
|
|
|
|
2018-04-24 07:16:39 +00:00
|
|
|
void FunctionModelEvaluate::executeImpl(Block & block, const ColumnNumbers & arguments, size_t result, size_t /*input_rows_count*/)
|
2017-10-17 10:44:46 +00:00
|
|
|
{
|
|
|
|
const auto name_col = checkAndGetColumnConst<ColumnString>(block.getByPosition(arguments[0]).column.get());
|
|
|
|
if (!name_col)
|
|
|
|
throw Exception("First argument of function " + getName() + " must be a constant string",
|
|
|
|
ErrorCodes::ILLEGAL_COLUMN);
|
|
|
|
|
|
|
|
auto model = models.getModel(name_col->getValue<String>());
|
|
|
|
|
2017-12-13 01:27:53 +00:00
|
|
|
ColumnRawPtrs columns;
|
2018-12-26 16:44:57 +00:00
|
|
|
Columns materialized_columns;
|
|
|
|
ColumnPtr null_map;
|
|
|
|
|
2017-10-20 09:52:36 +00:00
|
|
|
columns.reserve(arguments.size());
|
2018-12-26 16:44:57 +00:00
|
|
|
for (auto arg : ext::range(1, arguments.size()))
|
|
|
|
{
|
|
|
|
auto & column = block.getByPosition(arguments[arg]).column;
|
|
|
|
columns.push_back(column.get());
|
|
|
|
if (auto full_column = column->convertToFullColumnIfConst())
|
|
|
|
{
|
|
|
|
materialized_columns.push_back(full_column);
|
|
|
|
columns.back() = full_column.get();
|
|
|
|
}
|
|
|
|
if (auto * col_nullable = typeid_cast<const ColumnNullable *>(columns.back()))
|
|
|
|
{
|
|
|
|
if (!null_map)
|
|
|
|
null_map = col_nullable->getNullMapColumnPtr();
|
|
|
|
else
|
|
|
|
{
|
|
|
|
auto mut_null_map = (*std::move(null_map)).mutate();
|
|
|
|
|
|
|
|
NullMap & result_null_map = static_cast<ColumnUInt8 &>(*mut_null_map).getData();
|
|
|
|
const NullMap & src_null_map = col_nullable->getNullMapColumn().getData();
|
|
|
|
|
|
|
|
for (size_t i = 0, size = result_null_map.size(); i < size; ++i)
|
|
|
|
if (src_null_map[i])
|
|
|
|
result_null_map[i] = 1;
|
|
|
|
|
|
|
|
null_map = std::move(mut_null_map);
|
|
|
|
}
|
|
|
|
|
|
|
|
columns.back() = &col_nullable->getNestedColumn();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
auto res = model->evaluate(columns);
|
|
|
|
|
|
|
|
if (null_map)
|
|
|
|
{
|
|
|
|
if (auto * tuple = typeid_cast<const ColumnTuple *>(res.get()))
|
|
|
|
{
|
|
|
|
auto nested = tuple->getColumns();
|
|
|
|
for (auto & col : nested)
|
|
|
|
col = ColumnNullable::create(col, null_map);
|
|
|
|
|
|
|
|
res = ColumnTuple::create(nested);
|
|
|
|
}
|
|
|
|
else
|
|
|
|
res = ColumnNullable::create(res, null_map);
|
|
|
|
}
|
2017-10-17 10:44:46 +00:00
|
|
|
|
2018-12-26 16:44:57 +00:00
|
|
|
block.getByPosition(result).column = res;
|
2017-10-17 10:44:46 +00:00
|
|
|
}
|
|
|
|
|
2017-10-17 12:02:30 +00:00
|
|
|
void registerFunctionsExternalModels(FunctionFactory & factory)
|
|
|
|
{
|
|
|
|
factory.registerFunction<FunctionModelEvaluate>();
|
|
|
|
}
|
|
|
|
|
2017-10-17 10:44:46 +00:00
|
|
|
}
|