dbms: fixed error with faster text formats [#METR-20081].

This commit is contained in:
Alexey Milovidov 2016-03-03 04:54:58 +03:00
parent b5ac93cf0e
commit d38693267d
2 changed files with 122 additions and 200 deletions

View File

@ -105,6 +105,8 @@ public:
return dict_struct.attributes[&getAttribute(attribute_name) - attributes.data()].injective;
}
/// Во всех функциях ниже, key_columns должны быть полноценными (не константными) столбцами.
/// См. требование в IDataType.h для функций текстовой сериализации.
#define DECLARE(TYPE)\
void get##TYPE(\
const std::string & attribute_name, const ConstColumnPlainPtrs & key_columns, const DataTypes & key_types,\

View File

@ -784,23 +784,20 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!typeid_cast<const DataTypeUInt64 *>(arguments[1].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return new DataTypeUInt8;
}
@ -811,8 +808,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -824,8 +820,7 @@ private:
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -840,8 +835,7 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 2 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto id_col_untyped = block.getByPosition(arguments[1]).column.get();
if (const auto id_col = typeid_cast<const ColumnVector<UInt64> *>(id_col_untyped))
@ -865,8 +859,7 @@ private:
else
throw Exception{
"Second argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -883,15 +876,16 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 2 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto key_col_with_type = block.getByPosition(arguments[1]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
if (const ColumnTuple * key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
/// Функции у внешних словарей поддерживают только полноценные (не константные) столбцы с ключами.
const ColumnPtr key_col_materialized = key_col->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
@ -903,8 +897,7 @@ private:
else
throw Exception{
"Second argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -934,16 +927,14 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3 or 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
@ -951,8 +942,7 @@ private:
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
@ -961,8 +951,7 @@ private:
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (arguments.size() == 4 && !typeid_cast<const DataTypeDate *>(arguments[3].get()))
@ -970,8 +959,7 @@ private:
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return new DataTypeString;
@ -983,8 +971,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -997,8 +984,7 @@ private:
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1013,15 +999,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1046,8 +1030,7 @@ private:
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -1065,24 +1048,23 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const ColumnPtr key_col_materialized = key_col->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
@ -1094,8 +1076,7 @@ private:
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -1112,15 +1093,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1134,8 +1113,7 @@ private:
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -1164,8 +1142,7 @@ private:
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -1198,8 +1175,7 @@ private:
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -1228,22 +1204,19 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed " +
toString(arguments.size()) + ", should be 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName() +
", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
!typeid_cast<const DataTypeTuple *>(arguments[2].get()))
@ -1251,16 +1224,14 @@ private:
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeString *>(arguments[3].get()))
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName() +
", must be String.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
return new DataTypeString;
}
@ -1271,8 +1242,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -1284,8 +1254,7 @@ private:
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1300,15 +1269,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1320,8 +1287,7 @@ private:
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1357,8 +1323,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -1395,8 +1360,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -1411,23 +1375,24 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
const auto & key_col = typeid_cast<const ColumnTuple &>(*key_col_with_type.column);
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col.getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const ColumnPtr key_col_materialized = key_col.convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
const auto out = new ColumnString;
@ -1435,7 +1400,9 @@ private:
const auto default_col_untyped = block.getByPosition(arguments[3]).column.get();
if (const auto default_col = typeid_cast<const ColumnString *>(default_col_untyped))
{
dict->getString(attr_name, key_columns, key_types, default_col, out);
}
else if (const auto default_col = typeid_cast<const ColumnConst<String> *>(default_col_untyped))
{
const auto & def = default_col->getData();
@ -1444,8 +1411,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be String",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1532,16 +1498,14 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3 or 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
@ -1549,8 +1513,7 @@ private:
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
@ -1559,8 +1522,7 @@ private:
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (arguments.size() == 4 && !typeid_cast<const DataTypeDate *>(arguments[3].get()))
@ -1568,8 +1530,7 @@ private:
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be Date.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return new DataType;
@ -1581,8 +1542,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -1595,8 +1555,7 @@ private:
!executeDispatchRange<RangeHashedDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1611,15 +1570,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1646,8 +1603,7 @@ private:
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -1665,24 +1621,23 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 3 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
if (const auto key_col = typeid_cast<const ColumnTuple *>(key_col_with_type.column.get()))
{
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col->getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const ColumnPtr key_col_materialized = key_col->convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
@ -1696,8 +1651,7 @@ private:
else
throw Exception{
"Third argument of function " + getName() + " must be " + dict->getKeyDescription(),
ErrorCodes::TYPE_MISMATCH
};
ErrorCodes::TYPE_MISMATCH};
return true;
}
@ -1714,15 +1668,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1736,8 +1688,7 @@ private:
{
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -1776,8 +1727,7 @@ private:
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -1811,8 +1761,7 @@ private:
{
throw Exception{
"Fourth argument of function " + getName() + " must be Date",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
}
@ -1861,16 +1810,14 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 4.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeString *>(arguments[1].get()))
@ -1878,8 +1825,7 @@ private:
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()) &&
@ -1888,8 +1834,7 @@ private:
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64 or tuple(...).",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataType *>(arguments[3].get()))
@ -1897,8 +1842,7 @@ private:
throw Exception{
"Illegal type " + arguments[3]->getName() + " of fourth argument of function " + getName()
+ ", must be " + DataType{}.getName() + ".",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return new DataType;
@ -1910,8 +1854,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -1923,8 +1866,7 @@ private:
!executeDispatchComplex<ComplexKeyCacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -1939,15 +1881,13 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
@ -1959,8 +1899,7 @@ private:
else
throw Exception{
"Third argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -1999,8 +1938,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -2038,8 +1976,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
template <typename DictionaryType>
@ -2054,23 +1991,24 @@ private:
throw Exception{
"Function " + getName() + " for dictionary of type " + dict->getTypeName() +
" requires exactly 4 arguments",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
const auto attr_name_col = typeid_cast<const ColumnConst<String> *>(block.getByPosition(arguments[1]).column.get());
if (!attr_name_col)
throw Exception{
"Second argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
const auto & attr_name = attr_name_col->getData();
const auto key_col_with_type = block.getByPosition(arguments[2]);
const auto & key_col = typeid_cast<const ColumnTuple &>(*key_col_with_type.column);
const auto key_columns = ext::map<ConstColumnPlainPtrs>(key_col.getColumns(), [] (const ColumnPtr & ptr) {
return ptr.get();
});
const ColumnPtr key_col_materialized = key_col.convertToFullColumnIfConst();
const auto key_columns = ext::map<ConstColumnPlainPtrs>(
static_cast<const ColumnTuple &>(*key_col_materialized.get()).getColumns(), [](const ColumnPtr & ptr) { return ptr.get(); });
const auto & key_types = static_cast<const DataTypeTuple &>(*key_col_with_type.type).getElements();
/// @todo detect when all key columns are constant
@ -2096,8 +2034,7 @@ private:
else
throw Exception{
"Fourth argument of function " + getName() + " must be " + DataType{}.getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -2144,16 +2081,14 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[1].get()))
@ -2161,8 +2096,7 @@ private:
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return new DataTypeArray{new DataTypeUInt64};
@ -2174,8 +2108,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -2185,8 +2118,7 @@ private:
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -2200,8 +2132,7 @@ private:
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD
};
ErrorCodes::UNSUPPORTED_METHOD};
const auto get_hierarchies = [&] (const PODArray<UInt64> & in, PODArray<UInt64> & out, PODArray<UInt64> & offsets) {
const auto size = in.size();
@ -2283,8 +2214,7 @@ private:
{
throw Exception{
"Second argument of function " + getName() + " must be UInt64",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -2315,16 +2245,14 @@ private:
throw Exception{
"Number of arguments for function " + getName() + " doesn't match: passed "
+ toString(arguments.size()) + ", should be 3.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH
};
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH};
if (!typeid_cast<const DataTypeString *>(arguments[0].get()))
{
throw Exception{
"Illegal type " + arguments[0]->getName() + " of first argument of function " + getName()
+ ", expected a string.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[1].get()))
@ -2332,8 +2260,7 @@ private:
throw Exception{
"Illegal type " + arguments[1]->getName() + " of second argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
if (!typeid_cast<const DataTypeUInt64 *>(arguments[2].get()))
@ -2341,8 +2268,7 @@ private:
throw Exception{
"Illegal type " + arguments[2]->getName() + " of third argument of function " + getName()
+ ", must be UInt64.",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT
};
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
}
return new DataTypeUInt8;
@ -2354,8 +2280,7 @@ private:
if (!dict_name_col)
throw Exception{
"First argument of function " + getName() + " must be a constant string",
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
auto dict = dictionaries.getDictionary(dict_name_col->getData());
const auto dict_ptr = dict.get();
@ -2365,8 +2290,7 @@ private:
!executeDispatch<CacheDictionary>(block, arguments, result, dict_ptr))
throw Exception{
"Unsupported dictionary type " + dict_ptr->getTypeName(),
ErrorCodes::UNKNOWN_TYPE
};
ErrorCodes::UNKNOWN_TYPE};
}
template <typename DictionaryType>
@ -2380,8 +2304,7 @@ private:
if (!dict->hasHierarchy())
throw Exception{
"Dictionary does not have a hierarchy",
ErrorCodes::UNSUPPORTED_METHOD
};
ErrorCodes::UNSUPPORTED_METHOD};
const auto child_id_col_untyped = block.getByPosition(arguments[1]).column.get();
const auto ancestor_id_col_untyped = block.getByPosition(arguments[2]).column.get();
@ -2394,8 +2317,7 @@ private:
throw Exception{
"Illegal column " + child_id_col_untyped->getName()
+ " of second argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
return true;
}
@ -2437,8 +2359,7 @@ private:
throw Exception{
"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;
@ -2474,8 +2395,7 @@ private:
throw Exception{
"Illegal column " + ancestor_id_col_untyped->getName()
+ " of third argument of function " + getName(),
ErrorCodes::ILLEGAL_COLUMN
};
ErrorCodes::ILLEGAL_COLUMN};
}
return true;