mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-12-03 13:02:00 +00:00
clickhouse: fixed build [#CONV-6788].
This commit is contained in:
parent
64f4d743d2
commit
a375f43d63
@ -501,8 +501,6 @@ public:
|
||||
}
|
||||
|
||||
out_vec.resize(pos - begin);
|
||||
|
||||
return true;
|
||||
}
|
||||
else if(const ColumnConstString * col = dynamic_cast<const ColumnConstString *>(column))
|
||||
{
|
||||
@ -513,17 +511,101 @@ public:
|
||||
res = res.substr(0, pos - src.c_str());
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstString(col->size(), res);
|
||||
|
||||
return true;
|
||||
}
|
||||
else
|
||||
{
|
||||
return false;
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
/// Получает массив идентификаторов, состоящий из исходного и цепочки родителей.
|
||||
template <typename T, typename Transform, typename Dict, typename Name>
|
||||
class FunctionHierarchyWithDictionary : public IFunction
|
||||
{
|
||||
private:
|
||||
const SharedPtr<Dict> owned_dict;
|
||||
|
||||
public:
|
||||
FunctionHierarchyWithDictionary(const SharedPtr<Dict> & owned_dict_)
|
||||
: owned_dict(owned_dict_)
|
||||
{
|
||||
if (!owned_dict)
|
||||
throw Exception("Dictionaries was not loaded. You need to check configuration file.", ErrorCodes::DICTIONARIES_WAS_NOT_LOADED);
|
||||
}
|
||||
|
||||
/// Получить имя функции.
|
||||
String getName() const
|
||||
{
|
||||
return Name::get();
|
||||
}
|
||||
|
||||
/// Получить тип результата по типам аргументов. Если функция неприменима для данных аргументов - кинуть исключение.
|
||||
DataTypePtr getReturnType(const DataTypes & arguments) const
|
||||
{
|
||||
if (arguments.size() != 1)
|
||||
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
|
||||
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 1.",
|
||||
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
|
||||
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of argument of function " + getName(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
if (arguments[0]->getName() != TypeName<T>::get())
|
||||
throw Exception("Illegal type " + arguments[0]->getName() + " of argument of function " + getName()
|
||||
+ " (must be " + TypeName<T>::get() + ")",
|
||||
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
|
||||
|
||||
return new DataTypeArray(arguments[0]);
|
||||
}
|
||||
|
||||
/// Выполнить функцию над блоком.
|
||||
void execute(Block & block, const ColumnNumbers & arguments, size_t result)
|
||||
{
|
||||
if (const ColumnVector<T> * col_from = dynamic_cast<const ColumnVector<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
ColumnVector<T> * col_values = new ColumnVector<T>;
|
||||
ColumnArray * col_array = new ColumnArray(col_values);
|
||||
block.getByPosition(result).column = col_array;
|
||||
|
||||
ColumnArray::Offsets_t & res_offsets = col_array->getOffsets();
|
||||
typename ColumnVector<T>::Container_t & res_values = col_values->getData();
|
||||
|
||||
const typename ColumnVector<T>::Container_t & vec_from = col_from->getData();
|
||||
size_t size = vec_from.size();
|
||||
res_offsets.resize(size);
|
||||
res_values.reserve(size * 4);
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
for (size_t i = 0; i < size; ++i)
|
||||
{
|
||||
T cur = vec_from[i];
|
||||
while (cur)
|
||||
{
|
||||
res_values.push_back(cur);
|
||||
cur = Transform::toParent(cur, dict);
|
||||
}
|
||||
res_offsets[i] = res_values.size();
|
||||
}
|
||||
}
|
||||
else if (const ColumnConst<T> * col_from = dynamic_cast<const ColumnConst<T> *>(&*block.getByPosition(arguments[0]).column))
|
||||
{
|
||||
Array res;
|
||||
|
||||
const Dict & dict = *owned_dict;
|
||||
T cur = col_from->getData();
|
||||
while (cur)
|
||||
{
|
||||
res.push_back(static_cast<typename NearestFieldType<T>::Type>(cur));
|
||||
cur = Transform::toParent(cur, dict);
|
||||
}
|
||||
|
||||
block.getByPosition(result).column = new ColumnConstArray(col_from->size(), res, new typename DataTypeFromFieldType<T>::Type);
|
||||
}
|
||||
else
|
||||
throw Exception("Illegal column " + block.getByPosition(arguments[0]).column->getName()
|
||||
+ " of first argument of function " + Name::get(),
|
||||
ErrorCodes::ILLEGAL_COLUMN);
|
||||
}
|
||||
};
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user