提交 1d4d01a7 编写于 作者: A Alexey Milovidov

dbms: development [#CONV-2944].

上级 7620dfe4
......@@ -345,7 +345,7 @@ public:
DataTypePtr getReturnType(const DataTypes & arguments) const
{
if (arguments.size() != 2)
throw Exception("Number of arguments for function " + getName() + " doesn't match: passed "
throw Exception("Number of arguments for function '" + getName() + "' doesn't match: passed "
+ Poco::NumberFormatter::format(arguments.size()) + ", should be 2.",
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
......
......@@ -14,11 +14,13 @@ class ASTSet : public IAST
{
public:
SetPtr set;
String column_name;
ASTSet() {}
ASTSet(StringRange range_) : IAST(range_) {}
ASTSet(const String & column_name_) : column_name(column_name_) {}
ASTSet(StringRange range_, const String & column_name_) : IAST(range_), column_name(column_name_) {}
String getID() { return "Set"; }
ASTPtr clone() const { return new ASTSet(*this); }
String getColumnName() { return column_name; }
};
}
#include <set>
#include <boost/assign/list_inserter.hpp>
#include <DB/AggregateFunctions/AggregateFunctionCount.h>
#include <DB/AggregateFunctions/AggregateFunctionSum.h>
#include <DB/AggregateFunctions/AggregateFunctionAvg.h>
......@@ -152,17 +155,21 @@ AggregateFunctionPtr AggregateFunctionFactory::getByTypeID(const String & type_i
AggregateFunctionPtr AggregateFunctionFactory::tryGet(const String & name, const DataTypes & argument_types) const
{
try
{
return get(name, argument_types);
}
catch (const DB::Exception & e)
{
if (e.code() == ErrorCodes::UNKNOWN_AGGREGATE_FUNCTION)
return NULL;
else
throw;
}
std::set<String> names;
boost::assign::insert(names)
("count")
("any")
("anyLast")
("min")
("max")
("sum")
("avg")
("uniq");
return names.end() != names.find(name)
? get(name, argument_types)
: NULL;
}
......
......@@ -9,6 +9,9 @@
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/DataTypes/DataTypeSet.h>
#include <DB/Columns/ColumnSet.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/Expression.h>
......@@ -103,6 +106,8 @@ void Expression::addSemantic(ASTPtr & ast)
argument_types.push_back(arg->type);
else if (ASTLiteral * arg = dynamic_cast<ASTLiteral *>(&**it))
argument_types.push_back(arg->type);
else if (dynamic_cast<ASTSubquery *>(&**it) || dynamic_cast<ASTSet *>(&**it))
argument_types.push_back(new DataTypeSet);
}
node->aggregate_function = context.getAggregateFunctionsFactory().tryGet(node->name, argument_types);
......@@ -257,6 +262,16 @@ void Expression::executeImpl(ASTPtr ast, Block & block, unsigned part_id, bool o
column.type = node->type;
column.name = node->getColumnName();
block.insert(column);
}
else if (ASTSet * node = dynamic_cast<ASTSet *>(&*ast))
{
/// Множество в секции IN.
ColumnWithNameAndType column;
column.column = new ColumnSet(block.rows(), node->set);
column.type = new DataTypeSet;
column.name = node->getColumnName();
block.insert(column);
}
}
......@@ -491,12 +506,12 @@ void Expression::makeSetsImpl(ASTPtr ast)
* Это может быть перечисление значений или подзапрос.
* Перечисление значений парсится как функция tuple.
*/
ASTPtr arg = args.children[1];
ASTPtr & arg = args.children[1];
if (dynamic_cast<ASTSubquery *>(&*arg))
{
/// Исполняем подзапрос, превращаем результат в множество, и кладём это множество на место подзапроса.
InterpreterSelectQuery interpreter(arg, context, QueryProcessingStage::Complete);
ASTSet * ast_set = new ASTSet;
InterpreterSelectQuery interpreter(arg->children[0], context, QueryProcessingStage::Complete);
ASTSet * ast_set = new ASTSet(arg->getColumnName());
ast_set->set = new Set;
ast_set->set->create(interpreter.execute());
arg = ast_set;
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册