flink-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From fhueske <...@git.apache.org>
Subject [GitHub] flink pull request #4266: [FLINK-6232][Table&Sql] support proctime inner win...
Date Thu, 13 Jul 2017 10:10:41 GMT
Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/4266#discussion_r127178517
  
    --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamWindowJoin.scala
---
    @@ -0,0 +1,187 @@
    +/*
    + * 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._
    +import org.apache.calcite.rel.core.{JoinInfo, JoinRelType}
    +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter}
    +import org.apache.calcite.rex.RexNode
    +import org.apache.flink.api.java.functions.NullByteKeySelector
    +import org.apache.flink.streaming.api.datastream.DataStream
    +import org.apache.flink.table.api.{StreamQueryConfig, StreamTableEnvironment, TableException}
    +import org.apache.flink.table.plan.nodes.CommonJoin
    +import org.apache.flink.table.plan.schema.RowSchema
    +import org.apache.flink.table.runtime.join.{ProcTimeWindowInnerJoin, WindowJoinUtil}
    +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo}
    +import org.apache.flink.table.updateutils.UpdateCheckUtils
    +
    +/**
    +  * Flink RelNode which matches along with JoinOperator and its related operations.
    +  */
    +class DataStreamWindowJoin(
    +    cluster: RelOptCluster,
    +    traitSet: RelTraitSet,
    +    leftNode: RelNode,
    +    rightNode: RelNode,
    +    joinCondition: RexNode,
    +    joinType: JoinRelType,
    +    leftSchema: RowSchema,
    +    rightSchema: RowSchema,
    +    schema: RowSchema,
    +    isRowTime: Boolean,
    +    leftLowerBound: Long,
    +    leftUpperBound: Long,
    +    remainCondition: Option[RexNode],
    +    ruleDescription: String)
    +  extends BiRel(cluster, traitSet, leftNode, rightNode)
    +    with CommonJoin
    +    with DataStreamRel {
    +
    +  override def deriveRowType() = schema.logicalType
    +
    +  override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode
= {
    +    new DataStreamWindowJoin(
    +      cluster,
    +      traitSet,
    +      inputs.get(0),
    +      inputs.get(1),
    +      joinCondition,
    +      joinType,
    +      leftSchema,
    +      rightSchema,
    +      schema,
    +      isRowTime,
    +      leftLowerBound,
    +      leftUpperBound,
    +      remainCondition,
    +      ruleDescription)
    +  }
    +
    +  override def toString: String = {
    +    joinToString(
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def explainTerms(pw: RelWriter): RelWriter = {
    +    joinExplainTerms(
    +      super.explainTerms(pw),
    +      schema.logicalType,
    +      joinCondition,
    +      joinType,
    +      getExpressionString)
    +  }
    +
    +  override def translateToPlan(
    +      tableEnv: StreamTableEnvironment,
    +      queryConfig: StreamQueryConfig): DataStream[CRow] = {
    +
    +    val config = tableEnv.getConfig
    +
    +    val isLeftAppendOnly = UpdateCheckUtils.isAppendOnly(left)
    --- End diff --
    
    the ` UpdateCheckUtils.isAppendOnly` recursively checks if any downstream operator produces
updates. As soon as any downstream operator produces updates, the given operator has to be
able to handle them. 
    
    Updates can be encodes as retraction or be implicit per key-wise updates if the update
producing and receiving operators use the same keys. Retraction updates are encoded as two
messages. Non-retraction updates are encoded as single message and require a key to which
they relate (`CRow.change == true` -> insert or update per key, `CRow.change == false`
 -> delete on key). Right now, only UpsertTableSinks use non-retraction updates, but other
operators such as unbounded joins will use it as well.
    
    So even if `AccRetract` is false, the input might produce updates but those updates are
differently encoded, i.e., in a single message. The window stream join is not able to handle
updates (it ignores the `CRow.change` flag). Therefore, we must ensure that the inputs do
not produce updates.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message