[ 
https://issues.apache.org/jira/browse/FLINK-9344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16486934#comment-16486934
 ] 

ASF GitHub Bot commented on FLINK-9344:
---------------------------------------

Github user fhueske commented on a diff in the pull request:

    https://github.com/apache/flink/pull/5998#discussion_r190171657
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/runtime/setop/StreamIntersectCoProcessFunction.scala
 ---
    @@ -0,0 +1,236 @@
    +/*
    + * 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.setop
    +
    +import org.apache.flink.api.common.state.{ValueState, ValueStateDescriptor}
    +import org.apache.flink.api.common.typeinfo.TypeInformation
    +import org.apache.flink.api.java.tuple.{Tuple2 => JTuple2}
    +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.join.CRowWrappingMultiOutputCollector
    +import org.apache.flink.table.runtime.types.CRow
    +import 
org.apache.flink.table.typeutils.TypeCheckUtils.validateEqualsHashCode
    +import org.apache.flink.table.util.Logging
    +import org.apache.flink.types.Row
    +import org.apache.flink.util.Collector
    +
    +class StreamIntersectCoProcessFunction(
    +  resultType: TypeInformation[Row],
    +  queryConfig: StreamQueryConfig,
    +  all: Boolean)
    +  extends CoProcessFunction[CRow, CRow, CRow]
    --- End diff --
    
    I think it makes sense to have two implementations of this operator.
    1. For tables with a time attribute. This implementation works without 
retraction and can automatically cleanup the state. 
    2. For tables without time attributes. This implementation needs to cleanup 
state based on retention time and produces retractions.
    
    This PR seems to address both cases, which is fine for now. We can improve 
for 1. later on. Both cases should be implemented as `CoProcessFunction`. We 
should try to be independent of the DataStream window operators, IMO.


> Support INTERSECT and INTERSECT ALL for streaming
> -------------------------------------------------
>
>                 Key: FLINK-9344
>                 URL: https://issues.apache.org/jira/browse/FLINK-9344
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table API & SQL
>            Reporter: Ruidong Li
>            Assignee: Ruidong Li
>            Priority: Major
>
> support intersect and intersect all for both SQL and TableAPI



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to