clickhouse: fixed build [#CONV-6788].

This commit is contained in:
Michael Kolupaev 2013-03-07 12:47:12 +00:00
parent 64f4d743d2
commit a375f43d63

View File

@ -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);
}
};