Skip to content

[FLINK-12076] [table-planner-blink] Add support for generating optimized logical plan for simple group aggregate on batch #8092

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Merged
merged 4 commits into from
Apr 2, 2019
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,16 @@ public class PlannerConfigOptions {
"(only count RexCall node, including leaves and interior nodes). Negative number to" +
" use the default threshold: double of number of nodes.");

public static final ConfigOption<String> SQL_OPTIMIZER_AGG_PHASE_ENFORCER =
key("sql.optimizer.agg.phase.enforcer")
.defaultValue("NONE")
.withDescription("Strategy for agg phase. Only NONE, TWO_PHASE or ONE_PHASE can be set.\n" +
"NONE: No special enforcer for aggregate stage. Whether to choose two stage aggregate or one" +
" stage aggregate depends on cost. \n" +
"TWO_PHASE: Enforce to use two stage aggregate which has localAggregate and globalAggregate. " +
"NOTE: If aggregate call does not support split into two phase, still use one stage aggregate.\n" +
"ONE_PHASE: Enforce to use one stage aggregate which only has CompleteGlobalAggregate.");

public static final ConfigOption<Boolean> SQL_OPTIMIZER_SHUFFLE_PARTIAL_KEY_ENABLED =
key("sql.optimizer.shuffle.partial-key.enabled")
.defaultValue(false)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,8 +25,10 @@

import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.DIVIDE;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.EQUALS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.GREATER_THAN;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.IF;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.IS_NULL;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.LESS_THAN;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.MINUS;
import static org.apache.flink.table.expressions.BuiltInFunctionDefinitions.PLUS;

Expand Down Expand Up @@ -78,4 +80,12 @@ public static Expression div(Expression input1, Expression input2) {
public static Expression equalTo(Expression input1, Expression input2) {
return call(EQUALS, input1, input2);
}

public static Expression lessThan(Expression input1, Expression input2) {
return call(LESS_THAN, input1, input2);
}

public static Expression greaterThan(Expression input1, Expression input2) {
return call(GREATER_THAN, input1, input2);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,9 @@
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.UnresolvedFieldReferenceExpression;
import org.apache.flink.table.type.DecimalType;
import org.apache.flink.table.type.InternalType;
import org.apache.flink.table.type.InternalTypes;
import org.apache.flink.table.type.TypeConverters;
import org.apache.flink.table.typeutils.DecimalTypeInfo;

import java.math.BigDecimal;
Expand All @@ -44,9 +47,7 @@ public abstract class AvgAggFunction extends DeclarativeAggregateFunction {
private UnresolvedFieldReferenceExpression sum = new UnresolvedFieldReferenceExpression("sum");
private UnresolvedFieldReferenceExpression count = new UnresolvedFieldReferenceExpression("count");

public TypeInformation getSumType() {
return Types.LONG;
}
public abstract TypeInformation getSumType();

@Override
public int operandCount() {
Expand All @@ -60,6 +61,14 @@ public UnresolvedFieldReferenceExpression[] aggBufferAttributes() {
count};
}

@Override
public InternalType[] getAggBufferTypes() {
return new InternalType[] {
TypeConverters.createInternalTypeFromTypeInfo(getSumType()),
InternalTypes.LONG
};
}

@Override
public Expression[] initialValuesExpressions() {
return new Expression[] {
Expand Down Expand Up @@ -110,6 +119,11 @@ public static class IntegralAvgAggFunction extends AvgAggFunction {
public TypeInformation getResultType() {
return Types.DOUBLE;
}

@Override
public TypeInformation getSumType() {
return Types.LONG;
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.functions;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.UnresolvedFieldReferenceExpression;
import org.apache.flink.table.type.InternalType;
import org.apache.flink.table.type.InternalTypes;

import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
import static org.apache.flink.table.expressions.ExpressionBuilder.plus;

/**
* This count1 aggregate function returns the count1 of values
* which go into it like [[CountAggFunction]].
* It differs in that null values are also counted.
*/
public class Count1AggFunction extends DeclarativeAggregateFunction {
private UnresolvedFieldReferenceExpression count1 = new UnresolvedFieldReferenceExpression("count1");

@Override
public int operandCount() {
return 1;
}

@Override
public UnresolvedFieldReferenceExpression[] aggBufferAttributes() {
return new UnresolvedFieldReferenceExpression[] { count1 };
}

@Override
public InternalType[] getAggBufferTypes() {
return new InternalType[] { InternalTypes.LONG };
}

@Override
public TypeInformation getResultType() {
return Types.LONG;
}

@Override
public Expression[] initialValuesExpressions() {
return new Expression[] {
/* count1 = */ literal(0L, getResultType())
};
}

@Override
public Expression[] accumulateExpressions() {
return new Expression[] {
/* count1 = */ plus(count1, literal(1L))
};
}

@Override
public Expression[] retractExpressions() {
return new Expression[] {
/* count1 = */ minus(count1, literal(1L))
};
}

@Override
public Expression[] mergeExpressions() {
return new Expression[] {
/* count1 = */ plus(count1, mergeOperand(count1))
};
}

@Override
public Expression getValueExpression() {
return count1;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,93 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.flink.table.functions;

import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.api.common.typeinfo.Types;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.UnresolvedFieldReferenceExpression;
import org.apache.flink.table.type.InternalType;
import org.apache.flink.table.type.InternalTypes;

import static org.apache.flink.table.expressions.ExpressionBuilder.ifThenElse;
import static org.apache.flink.table.expressions.ExpressionBuilder.isNull;
import static org.apache.flink.table.expressions.ExpressionBuilder.literal;
import static org.apache.flink.table.expressions.ExpressionBuilder.minus;
import static org.apache.flink.table.expressions.ExpressionBuilder.plus;

/**
* built-in count aggregate function.
*/
public class CountAggFunction extends DeclarativeAggregateFunction {
private UnresolvedFieldReferenceExpression count = new UnresolvedFieldReferenceExpression("count");

@Override
public int operandCount() {
return 1;
}

@Override
public UnresolvedFieldReferenceExpression[] aggBufferAttributes() {
return new UnresolvedFieldReferenceExpression[] { count };
}

@Override
public InternalType[] getAggBufferTypes() {
return new InternalType[] { InternalTypes.LONG };
}

@Override
public TypeInformation getResultType() {
return Types.LONG;
}

@Override
public Expression[] initialValuesExpressions() {
return new Expression[] {
/* count = */ literal(0L, getResultType())
};
}

@Override
public Expression[] accumulateExpressions() {
return new Expression[] {
/* count = */ ifThenElse(isNull(operand(0)), count, plus(count, literal(1L)))
};
}

@Override
public Expression[] retractExpressions() {
return new Expression[] {
/* count = */ ifThenElse(isNull(operand(0)), count, minus(count, literal(1L)))
};
}

@Override
public Expression[] mergeExpressions() {
return new Expression[] {
/* count = */ plus(count, mergeOperand(count))
};
}

// If all input are nulls, count will be 0 and we will get result 0.
@Override
public Expression getValueExpression() {
return count;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.flink.api.common.typeinfo.TypeInformation;
import org.apache.flink.table.expressions.Expression;
import org.apache.flink.table.expressions.UnresolvedFieldReferenceExpression;
import org.apache.flink.table.type.InternalType;
import org.apache.flink.util.Preconditions;

import java.util.Arrays;
Expand Down Expand Up @@ -57,6 +58,11 @@ public abstract class DeclarativeAggregateFunction extends UserDefinedFunction {
*/
public abstract UnresolvedFieldReferenceExpression[] aggBufferAttributes();

/**
* All types of the aggregate buffer.
*/
public abstract InternalType[] getAggBufferTypes();

/**
* The result type of the function.
*/
Expand Down
Loading