[ https://issues.apache.org/jira/browse/FLINK-6094?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16264514#comment-16264514 ]
ASF GitHub Bot commented on FLINK-6094: --------------------------------------- Github user twalthr commented on a diff in the pull request: https://github.com/apache/flink/pull/4471#discussion_r152760126 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/join/DataStreamInnerJoin.scala --- @@ -0,0 +1,285 @@ +/* + * 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.runtime.join + +import org.apache.flink.api.common.functions.FlatJoinFunction +import org.apache.flink.api.common.state._ +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.typeutils.TupleTypeInfo +import org.apache.flink.configuration.Configuration +import org.apache.flink.streaming.api.functions.co.CoProcessFunction +import org.apache.flink.table.api.{StreamQueryConfig, Types} +import org.apache.flink.table.runtime.CRowWrappingMultiOuputCollector +import org.apache.flink.table.runtime.types.CRow +import org.apache.flink.types.Row +import org.apache.flink.util.Collector +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2} +import org.slf4j.LoggerFactory +import org.apache.flink.table.codegen.Compiler + + +/** + * Connect data for left stream and right stream. Only use for innerJoin. + * + * @param leftType the input type of left stream + * @param rightType the input type of right stream + * @param resultType the output type of join + * @param genJoinFuncName the function code of other non-equi condition + * @param genJoinFuncCode the function name of other non-equi condition + * @param queryConfig the configuration for the query to generate + */ +class DataStreamInnerJoin( + leftType: TypeInformation[Row], + rightType: TypeInformation[Row], + resultType: TypeInformation[CRow], + genJoinFuncName: String, + genJoinFuncCode: String, + queryConfig: StreamQueryConfig) + extends CoProcessFunction[CRow, CRow, CRow] + with Compiler[FlatJoinFunction[Row, Row, Row]] { + + // state to hold left stream element + private var leftState: MapState[Row, JTuple2[Int, Long]] = _ + // state to hold right stream element + private var rightState: MapState[Row, JTuple2[Int, Long]] = _ + private var cRowWrapper: CRowWrappingMultiOuputCollector = _ + + private val minRetentionTime: Long = queryConfig.getMinIdleStateRetentionTime + private val maxRetentionTime: Long = queryConfig.getMaxIdleStateRetentionTime + private val stateCleaningEnabled: Boolean = minRetentionTime > 1 + + // state to record last timer of left stream, 0 means no timer + private var leftTimer: ValueState[Long] = _ + // state to record last timer of right stream, 0 means no timer + private var rightTimer: ValueState[Long] = _ + + // other condition function + private var joinFunction: FlatJoinFunction[Row, Row, Row] = _ + + val LOG = LoggerFactory.getLogger(this.getClass) --- End diff -- Use the new `with Logging` interface. > Implement stream-stream proctime non-window inner join > ------------------------------------------------------- > > Key: FLINK-6094 > URL: https://issues.apache.org/jira/browse/FLINK-6094 > Project: Flink > Issue Type: New Feature > Components: Table API & SQL > Reporter: Shaoxuan Wang > Assignee: Hequn Cheng > > This includes: > 1.Implement stream-stream proctime non-window inner join > 2.Implement the retract process logic for join -- This message was sent by Atlassian JIRA (v6.4.14#64029)