[
https://issues.apache.org/jira/browse/FLINK-10977?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16718846#comment-16718846
]
ASF GitHub Bot commented on FLINK-10977:
----------------------------------------
dianfu commented on a change in pull request #7209: [FLINK-10977][table] Add
UnBounded FlatAggregate operator to streaming Table API
URL: https://github.com/apache/flink/pull/7209#discussion_r240958352
##########
File path:
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamTableAggregate.scala
##########
@@ -0,0 +1,148 @@
+/*
+ * 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.plan.nodes.datastream
+
+import org.apache.calcite.plan.{RelOptCluster, RelTraitSet}
+import org.apache.calcite.rel.core.AggregateCall
+import org.apache.calcite.rel.{RelNode, RelWriter, SingleRel}
+import org.apache.flink.api.java.functions.NullByteKeySelector
+import org.apache.flink.streaming.api.datastream.DataStream
+import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment}
+import org.apache.flink.table.functions.utils.TableAggSqlFunction
+import org.apache.flink.table.plan.nodes.CommonAggregate
+import org.apache.flink.table.plan.rules.datastream.DataStreamRetractionRules
+import org.apache.flink.table.plan.schema.RowSchema
+import org.apache.flink.table.runtime.CRowKeySelector
+import org.apache.flink.table.runtime.aggregate.AggregateUtil.CalcitePair
+import org.apache.flink.table.runtime.aggregate._
+import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
+import org.apache.flink.table.util.Logging
+
+/**
+ *
+ * Flink RelNode for data stream unbounded table aggregate
+ *
+ * @param cluster Cluster of the RelNode, represent for an
environment of related
+ * relational expressions during the optimization of a
query.
+ * @param traitSet Trait set of the RelNode
+ * @param inputNode The input RelNode of aggregation
+ * @param schema The type of the rows emitted by this RelNode
+ * @param inputSchema The type of the rows consumed by this RelNode
+ * @param namedAggregates List of calls to aggregate functions and their
output field names
+ * @param groupings The position (in the input Row) of the grouping keys
+ */
+class DataStreamTableAggregate(
+ cluster: RelOptCluster,
+ traitSet: RelTraitSet,
+ inputNode: RelNode,
+ schema: RowSchema,
+ inputSchema: RowSchema,
+ val namedAggregates: Seq[CalcitePair[AggregateCall, String]],
+ val groupings: Array[Int])
+ extends SingleRel(cluster, traitSet, inputNode)
+ with CommonAggregate
+ with DataStreamRel
+ with Logging {
+
+ override def deriveRowType() = schema.relDataType
+
+ override def needsUpdatesAsRetraction = true
+
+ override def producesUpdates = true
+
+ override def consumesRetractions = true
+
+ override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]):
RelNode = {
+ new DataStreamTableAggregate(
+ cluster,
+ traitSet,
+ inputs.get(0),
+ schema,
+ inputSchema,
+ namedAggregates,
+ groupings)
+ }
+
+ override def toString: String = {
+ s"TableAggregate(${
+ if (!groupings.isEmpty) {
+ s"groupBy: (${groupingToString(inputSchema.relDataType, groupings)}), "
+ } else {
+ ""
+ }
+ } flatAggregate:(${aggregationToString(
+ inputSchema.relDataType, groupings, getRowType, namedAggregates, Nil)}))"
+ }
+
+ override def explainTerms(pw: RelWriter): RelWriter = {
+ super.explainTerms(pw)
+ .itemIf("groupBy", groupingToString(
+ inputSchema.relDataType, groupings), !groupings.isEmpty)
+ .item("flatAggregate", aggregationToString(
+ inputSchema.relDataType, groupings, getRowType, namedAggregates, Nil))
+ }
+
+ override def translateToPlan(
+ tableEnv: StreamTableEnvironment,
+ queryConfig: StreamQueryConfig): DataStream[CRow] = {
+
+ val inputDS =
getInput.asInstanceOf[DataStreamRel].translateToPlan(tableEnv, queryConfig)
+ val outRowType = CRowTypeInfo(schema.typeInfo)
+ val aggCall = namedAggregates(0).left
Review comment:
namedAggregates(0) -> namedAggregates.head
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
For queries about this service, please contact Infrastructure at:
[email protected]
> Add FlatAggregate operator to unbounded streaming Table API
> -----------------------------------------------------------
>
> Key: FLINK-10977
> URL: https://issues.apache.org/jira/browse/FLINK-10977
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Reporter: sunjincheng
> Assignee: Hequn Cheng
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.8.0
>
>
> Add FlatAggregate operator to streaming Table API as described in [Google
> doc|https://docs.google.com/document/d/1tnpxg31EQz2-MEzSotwFzqatsB4rNLz0I-l_vPa5H4Q/edit#heading=h.q23rny2iglsr].
> The usage:
> {code:java}
> val res = tab
> .groupBy('a) // leave out groupBy-clause to define global table aggregates
> .flatAgg(fun: TableAggregateFunction) // output has columns 'a, 'b, 'c
> .select('a, 'c){code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)