Skip to content

Commit

Permalink
[FLINK-23544][table-planner] Window TVF Supports session window in plan
Browse files Browse the repository at this point in the history
This closes apache#16669
  • Loading branch information
beyond1920 authored and godfreyhe committed Aug 11, 2021
1 parent 3921231 commit 34d5100
Show file tree
Hide file tree
Showing 19 changed files with 1,004 additions and 21 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1179,4 +1179,5 @@ public List<SqlGroupedWindowFunction> getAuxiliaryFunctions() {
public static final SqlFunction TUMBLE = new SqlTumbleTableFunction();
public static final SqlFunction HOP = new SqlHopTableFunction();
public static final SqlFunction CUMULATE = new SqlCumulateTableFunction();
public static final SqlFunction SESSION = new SqlSessionTableFunction();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,67 @@
/*
* 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.planner.functions.sql;

import org.apache.flink.shaded.guava30.com.google.common.collect.ImmutableList;

import org.apache.calcite.sql.SqlCallBinding;
import org.apache.calcite.sql.SqlKind;
import org.apache.calcite.sql.SqlOperator;

/**
* SqlSessionTableFunction implements an operator for session.
*
* <p>It allows three parameters:
*
* <ol>
* <li>a table
* <li>a descriptor to provide a time attribute column name from the input table
* <li>an interval parameter to specify an inactive activity gap to break sessions
* </ol>
*/
public class SqlSessionTableFunction extends SqlWindowTableFunction {

public SqlSessionTableFunction() {
super(SqlKind.SESSION.name(), new OperandMetadataImpl());
}

/** Operand type checker for SESSION. */
private static class OperandMetadataImpl extends AbstractOperandMetadata {
OperandMetadataImpl() {
super(ImmutableList.of(PARAM_DATA, PARAM_TIMECOL, PARAM_SESSION_GAP), 3);
}

@Override
public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
if (!checkTableAndDescriptorOperands(callBinding, 1)) {
return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
}
if (!checkIntervalOperands(callBinding, 2)) {
return throwValidationSignatureErrorOrReturnFalse(callBinding, throwOnFailure);
}
// check time attribute
return throwExceptionOrReturnFalse(
checkTimeColumnDescriptorOperand(callBinding, 1), throwOnFailure);
}

@Override
public String getAllowedSignatures(SqlOperator op, String opName) {
return opName + "(TABLE table_name, DESCRIPTOR(timecol), datetime interval)";
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -83,6 +83,9 @@ public class SqlWindowTableFunction extends SqlFunction implements SqlTableFunct
/** The slide interval, only used for HOP window. */
protected static final String PARAM_STEP = "STEP";

/** The session gap interval, only used for SESSION window. */
protected static final String PARAM_SESSION_GAP = "GAP";

/**
* Type-inference strategy whereby the row type of a table function call is a ROW, which is
* combined from the row type of operand #0 (which is a TABLE) and two additional fields. The
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* 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.planner.plan.logical;

import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonCreator;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.flink.shaded.jackson2.com.fasterxml.jackson.annotation.JsonTypeName;

import java.time.Duration;
import java.util.Objects;

import static org.apache.flink.util.Preconditions.checkNotNull;
import static org.apache.flink.util.TimeUtils.formatWithHighestUnit;

/** Logical representation of a session window specification. */
@JsonTypeName("SessionWindow")
public class SessionWindowSpec implements WindowSpec {
public static final String FIELD_NAME_GAP = "gap";

@JsonProperty(FIELD_NAME_GAP)
private final Duration gap;

@JsonCreator
public SessionWindowSpec(@JsonProperty(FIELD_NAME_GAP) Duration gap) {
this.gap = checkNotNull(gap);
}

@Override
public String toSummaryString(String windowing) {
return String.format("SESSION(%s, gap=[%s])", windowing, formatWithHighestUnit(gap));
}

public Duration getGap() {
return gap;
}

@Override
public boolean equals(Object o) {
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
SessionWindowSpec that = (SessionWindowSpec) o;
return gap.equals(that.gap);
}

@Override
public int hashCode() {
return Objects.hash(SessionWindowSpec.class, gap);
}

@Override
public String toString() {
return String.format("SESSION(gap=[%s])", formatWithHighestUnit(gap));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,8 @@
@JsonSubTypes({
@JsonSubTypes.Type(value = TumblingWindowSpec.class),
@JsonSubTypes.Type(value = HoppingWindowSpec.class),
@JsonSubTypes.Type(value = CumulativeWindowSpec.class)
@JsonSubTypes.Type(value = CumulativeWindowSpec.class),
@JsonSubTypes.Type(value = SessionWindowSpec.class)
})
public interface WindowSpec {

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import org.apache.flink.table.planner.delegation.PlannerBase;
import org.apache.flink.table.planner.plan.logical.CumulativeWindowSpec;
import org.apache.flink.table.planner.plan.logical.HoppingWindowSpec;
import org.apache.flink.table.planner.plan.logical.SessionWindowSpec;
import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
import org.apache.flink.table.planner.plan.logical.TumblingWindowSpec;
import org.apache.flink.table.planner.plan.logical.WindowSpec;
Expand Down Expand Up @@ -119,6 +120,10 @@ protected Transformation<RowData> translateToPlanInternal(PlannerBase planner) {
+ "3. join with join condition contains window starts equality of input tables "
+ "and window ends equality of input tables.\n",
windowSummary));
} else if (windowingStrategy.getWindow() instanceof SessionWindowSpec) {
// WindowTableFunctionOperator is not suitable for Session Window because can't do
// state-less window assigning for input row per record for Session Window.
throw new TableException("Session Window TableFunction is not supported yet.");
} else if (!windowingStrategy.isRowtime()) {
throw new TableException("Processing time Window TableFunction is not supported yet.");
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.flink.table.planner.plan.rules.physical.stream;

import org.apache.flink.table.api.TableConfig;
import org.apache.flink.table.planner.plan.logical.SessionWindowSpec;
import org.apache.flink.table.planner.plan.logical.SliceAttachedWindowingStrategy;
import org.apache.flink.table.planner.plan.logical.TimeAttributeWindowingStrategy;
import org.apache.flink.table.planner.plan.logical.WindowAttachedWindowingStrategy;
Expand Down Expand Up @@ -95,6 +96,12 @@ public boolean matches(RelOptRuleCall call) {
return false;
}

// session window doesn't support two-phase,
// otherwise window assigner results may be different
if (windowing.getWindow() instanceof SessionWindowSpec) {
return false;
}

// all aggregate function should support merge() method
if (!AggregateUtil.doAllSupportPartialMerge(windowAgg.aggInfoList().aggInfos())) {
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.flink.table.planner.plan.nodes.physical.stream

import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.expressions.{PlannerNamedWindowProperty, PlannerSliceEnd, PlannerWindowReference}
import org.apache.flink.table.planner.plan.logical.{TimeAttributeWindowingStrategy, WindowAttachedWindowingStrategy, WindowingStrategy}
import org.apache.flink.table.planner.plan.logical.{SessionWindowSpec, TimeAttributeWindowingStrategy, WindowAttachedWindowingStrategy, WindowingStrategy}
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecLocalWindowAggregate
import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
import org.apache.flink.table.planner.plan.rules.physical.stream.TwoStageOptimizedWindowAggregateRule
Expand Down Expand Up @@ -69,8 +69,17 @@ class StreamPhysicalLocalWindowAggregate(

override def isValid(litmus: Litmus, context: RelNode.Context): Boolean = {
windowing match {
case _: WindowAttachedWindowingStrategy | _: TimeAttributeWindowingStrategy =>
case _: WindowAttachedWindowingStrategy =>
// pass
case tws: TimeAttributeWindowingStrategy =>
tws.getWindow match {
case _: SessionWindowSpec =>
return litmus.fail("StreamPhysicalLocalWindowAggregate should not accept " +
"TimeAttributeWindowingStrategy with Session window. " +
"This should never happen, please open an issue.")
case _ =>
// pass
}
case _ =>
return litmus.fail("StreamPhysicalLocalWindowAggregate should only accepts " +
"WindowAttachedWindowingStrategy and TimeAttributeWindowingStrategy, " +
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,22 +18,23 @@

package org.apache.flink.table.planner.plan.nodes.physical.stream

import org.apache.flink.table.expressions.ApiExpressionUtils.intervalOfMillis
import org.apache.flink.table.expressions.FieldReferenceExpression
import org.apache.flink.table.planner.calcite.FlinkTypeFactory
import org.apache.flink.table.planner.expressions.{PlannerNamedWindowProperty, PlannerProctimeAttribute, PlannerRowtimeAttribute, PlannerWindowEnd, PlannerWindowStart}
import org.apache.flink.table.planner.plan.logical.WindowingStrategy
import org.apache.flink.table.planner.plan.nodes.exec.stream.StreamExecWindowAggregate
import org.apache.flink.table.planner.expressions.{PlannerNamedWindowProperty, PlannerWindowReference}
import org.apache.flink.table.planner.plan.logical.{SessionGroupWindow, SessionWindowSpec, TimeAttributeWindowingStrategy, WindowingStrategy}
import org.apache.flink.table.planner.plan.nodes.exec.stream.{StreamExecGroupWindowAggregate, StreamExecWindowAggregate}
import org.apache.flink.table.planner.plan.nodes.exec.{ExecNode, InputProperty}
import org.apache.flink.table.planner.plan.utils.{AggregateInfoList, AggregateUtil, FlinkRelOptUtil, RelExplainUtil, WindowUtil}
import org.apache.flink.table.planner.plan.utils.WindowUtil.checkEmitConfiguration
import org.apache.flink.table.types.logical.utils.LogicalTypeUtils
import org.apache.flink.table.runtime.types.LogicalTypeDataTypeConverter.fromLogicalTypeToDataType

import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
import org.apache.calcite.rel.`type`.RelDataType
import org.apache.calcite.rel.core.{Aggregate, AggregateCall}
import org.apache.calcite.rel.core.AggregateCall
import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
import org.apache.calcite.util.ImmutableBitSet

import java.util
import java.util.Collections

import scala.collection.JavaConverters._

Expand Down Expand Up @@ -104,14 +105,45 @@ class StreamPhysicalWindowAggregate(

override def translateToExecNode(): ExecNode[_] = {
checkEmitConfiguration(FlinkRelOptUtil.getTableConfigFromContext(this))
new StreamExecWindowAggregate(
grouping,
aggCalls.toArray,
windowing,
namedWindowProperties.toArray,
InputProperty.DEFAULT,
FlinkTypeFactory.toLogicalRowType(getRowType),
getRelDetailedDescription
)
windowing.getWindow match {
case windowSpec: SessionWindowSpec =>
windowing match {
case timeWindowStrategy: TimeAttributeWindowingStrategy =>
val timeAttributeFieldName = getInput.getRowType.getFieldNames.get(
timeWindowStrategy.getTimeAttributeIndex)
val timeAttributeType = windowing.getTimeAttributeType
val logicalWindow = SessionGroupWindow(
new PlannerWindowReference("w$", timeAttributeType),
new FieldReferenceExpression(
timeAttributeFieldName,
fromLogicalTypeToDataType(timeAttributeType),
0,
timeWindowStrategy.getTimeAttributeIndex),
intervalOfMillis(windowSpec.getGap.toMillis)
)
new StreamExecGroupWindowAggregate(
grouping,
aggCalls.toArray,
logicalWindow,
namedWindowProperties.toArray,
false,
InputProperty.DEFAULT,
FlinkTypeFactory.toLogicalRowType(getRowType),
getRelDetailedDescription
)
case _ =>
throw new UnsupportedOperationException(s"$windowing is not supported yet.")
}
case _ =>
new StreamExecWindowAggregate(
grouping,
aggCalls.toArray,
windowing,
namedWindowProperties.toArray,
InputProperty.DEFAULT,
FlinkTypeFactory.toLogicalRowType(getRowType),
getRelDetailedDescription
)
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import org.apache.flink.table.api.TableException
import org.apache.flink.table.api.config.OptimizerConfigOptions
import org.apache.flink.table.planner.calcite.{FlinkContext, FlinkLogicalRelFactories, FlinkRelBuilder}
import org.apache.flink.table.planner.functions.sql.{FlinkSqlOperatorTable, SqlFirstLastValueAggFunction}
import org.apache.flink.table.planner.plan.logical.SessionWindowSpec
import org.apache.flink.table.planner.plan.PartialFinalType
import org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery
import org.apache.flink.table.planner.plan.nodes.FlinkRelNode
Expand Down Expand Up @@ -138,11 +139,18 @@ class SplitAggregateRule extends RelOptRule(
val windowProps = fmq.getRelWindowProperties(agg.getInput)
val isWindowAgg = WindowUtil.groupingContainsWindowStartEnd(agg.getGroupSet, windowProps)
val isProctimeWindowAgg = isWindowAgg && !windowProps.isRowtime

// disable distinct split for session window,
// otherwise window assigner results may be different
val isSessionWindowAgg = isWindowAgg &&
windowProps.getWindowSpec.isInstanceOf[SessionWindowSpec]

// TableAggregate is not supported. see also FLINK-21923.
val isTableAgg = AggregateUtil.isTableAggregate(agg.getAggCallList)

agg.partialFinalType == PartialFinalType.NONE && agg.containsDistinctCall() &&
splitDistinctAggEnabled && isAllAggSplittable && !isProctimeWindowAgg && !isTableAgg
splitDistinctAggEnabled && isAllAggSplittable && !isProctimeWindowAgg &&
!isTableAgg && !isSessionWindowAgg
}

override def onMatch(call: RelOptRuleCall): Unit = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -218,6 +218,10 @@ object WindowUtil {
val step = getOperandAsLong(windowCall.operands(2))
val maxSize = getOperandAsLong(windowCall.operands(3))
new CumulativeWindowSpec(Duration.ofMillis(maxSize), Duration.ofMillis(step), offset)

case FlinkSqlOperatorTable.SESSION =>
val gap = getOperandAsLong(windowCall.operands(2))
new SessionWindowSpec(Duration.ofMillis(gap))
}

new TimeAttributeWindowingStrategy(windowSpec, timeAttributeType, timeIndex)
Expand Down
Loading

0 comments on commit 34d5100

Please sign in to comment.