topAggNode = new AggregatePlanNode();
}
}
int outputColumnIndex = 0;
NodeSchema agg_schema = new NodeSchema();
NodeSchema top_agg_schema = new NodeSchema();
for (ParsedSelectStmt.ParsedColInfo col : m_parsedSelect.m_aggResultColumns) {
AbstractExpression rootExpr = col.expression;
AbstractExpression agg_input_expr = null;
SchemaColumn schema_col = null;
SchemaColumn top_schema_col = null;
if (rootExpr instanceof AggregateExpression) {
ExpressionType agg_expression_type = rootExpr.getExpressionType();
agg_input_expr = rootExpr.getLeft();
// A bit of a hack: ProjectionNodes after the
// aggregate node need the output columns here to
// contain TupleValueExpressions (effectively on a temp table).
// So we construct one based on the output of the
// aggregate expression, the column alias provided by HSQL,
// and the offset into the output table schema for the
// aggregate node that we're computing.
// Oh, oh, it's magic, you know..
TupleValueExpression tve = new TupleValueExpression(
"VOLT_TEMP_TABLE", "VOLT_TEMP_TABLE", "", col.alias, outputColumnIndex);
tve.setTypeSizeBytes(rootExpr.getValueType(), rootExpr.getValueSize(),
rootExpr.getInBytes());
boolean is_distinct = ((AggregateExpression)rootExpr).isDistinct();
aggNode.addAggregate(agg_expression_type, is_distinct, outputColumnIndex, agg_input_expr);
schema_col = new SchemaColumn("VOLT_TEMP_TABLE", "VOLT_TEMP_TABLE", "", col.alias, tve);
top_schema_col = new SchemaColumn("VOLT_TEMP_TABLE", "VOLT_TEMP_TABLE", "", col.alias, tve);
/*
* Special case count(*), count(), sum(), min() and max() to
* push them down to each partition. It will do the
* push-down if the select columns only contains the listed
* aggregate operators and other group-by columns. If the
* select columns includes any other aggregates, it will not
* do the push-down. - nshi
*/
if (topAggNode != null) {
ExpressionType top_expression_type = agg_expression_type;
/*
* For count(*), count() and sum(), the pushed-down
* aggregate node doesn't change. An extra sum()
* aggregate node is added to the coordinator to sum up
* the numbers from all the partitions. The input schema
* and the output schema of the sum() aggregate node is
* the same as the output schema of the push-down
* aggregate node.
*
* If DISTINCT is specified, don't do push-down for
* count() and sum()
*/
if (agg_expression_type == ExpressionType.AGGREGATE_COUNT_STAR ||
agg_expression_type == ExpressionType.AGGREGATE_COUNT ||
agg_expression_type == ExpressionType.AGGREGATE_SUM) {
if (is_distinct) {
topAggNode = null;
}
else {
top_expression_type = ExpressionType.AGGREGATE_SUM;
}
}
/*
* For min() and max(), the pushed-down aggregate node
* doesn't change. An extra aggregate node of the same
* type is added to the coordinator. The input schema
* and the output schema of the top aggregate node is
* the same as the output schema of the pushed-down
* aggregate node.
*/
else if (agg_expression_type != ExpressionType.AGGREGATE_MIN &&
agg_expression_type != ExpressionType.AGGREGATE_MAX) {
/*
* Unsupported aggregate for push-down (AVG for example).
*/
topAggNode = null;
}
if (topAggNode != null) {
/*
* Input column of the top aggregate node is the output column of the push-down aggregate node
*/
topAggNode.addAggregate(top_expression_type, is_distinct, outputColumnIndex, tve);
}
}
}
// If the rootExpr is not itself an AggregateExpression but simply contains one (or more)
// like "MAX(counter)+1" or "MAX(col)/MIN(col)" the assumptions about matching input and output
// columns break down.
else if (rootExpr.hasAnySubexpressionOfClass(AggregateExpression.class)) {
assert(false);
}
else
{
/*
* These columns are the pass through columns that are not being
* aggregated on. These are the ones from the SELECT list. They
* MUST already exist in the child node's output. Find them and
* add them to the aggregate's output.
*/
schema_col = new SchemaColumn(col.tableName, col.tableAlias, col.columnName, col.alias, col.expression);
AbstractExpression topExpr = null;
if (col.groupBy) {
topExpr = m_parsedSelect.m_groupByExpressions.get(col.alias);
} else {
topExpr = col.expression;
}
top_schema_col = new SchemaColumn(col.tableName, col.tableAlias, col.columnName, col.alias, topExpr);
}
agg_schema.addColumn(schema_col);
top_agg_schema.addColumn(top_schema_col);
outputColumnIndex++;
}
for (ParsedSelectStmt.ParsedColInfo col : m_parsedSelect.m_groupByColumns) {
aggNode.addGroupByExpression(col.expression);