Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 0018A200CE1 for ; Thu, 10 Aug 2017 16:34:17 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id F2E0916B69E; Thu, 10 Aug 2017 14:34:17 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 374C816B69F for ; Thu, 10 Aug 2017 16:34:17 +0200 (CEST) Received: (qmail 39469 invoked by uid 500); 10 Aug 2017 14:34:16 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 38970 invoked by uid 99); 10 Aug 2017 14:34:15 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 10 Aug 2017 14:34:15 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id DDBFB18081A for ; Thu, 10 Aug 2017 14:34:14 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id jUKgBcubs8r5 for ; Thu, 10 Aug 2017 14:34:13 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id B503A5FAF3 for ; Thu, 10 Aug 2017 14:34:10 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 6FADDE0EA9 for ; Thu, 10 Aug 2017 14:34:08 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id DF92F241D9 for ; Thu, 10 Aug 2017 14:34:05 +0000 (UTC) Date: Thu, 10 Aug 2017 14:34:05 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-6094) Implement stream-stream proctime non-window inner join MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 10 Aug 2017 14:34:18 -0000 [ https://issues.apache.org/jira/browse/FLINK-6094?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16121694#comment-16121694 ] ASF GitHub Bot commented on FLINK-6094: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/4471#discussion_r132277678 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/util/UpdatingPlanChecker.scala --- @@ -90,40 +96,86 @@ object UpdatingPlanChecker { // resolve names of input fields .map(io => (inNames.get(io._1), io._2)) - // filter by input keys - val outKeys = inOutNames.filter(io => keys.get.contains(io._1)).map(_._2) - // check if all keys have been preserved - if (outKeys.nonEmpty && outKeys.length == keys.get.length) { + // filter by input keyAncestors + val outKeyAncesters = inOutNames + .filter(io => keyAncestors.get.map(e => e._1).contains(io._1)) + .map(io => (io._2, keyAncestors.get.find(ka => ka._1 == io._1).get._2)) + + // check if all keyAncestors have been preserved + if (outKeyAncesters.nonEmpty && + outKeyAncesters.map(ka => ka._2).distinct.length == + keyAncestors.get.map(ka => ka._2).distinct.length) { // all key have been preserved (but possibly renamed) - keys = Some(outKeys.toArray) + Some(outKeyAncesters.toList) } else { // some (or all) keys have been removed. Keys are no longer unique and removed - keys = None + None } + } else { + None } + case _: DataStreamOverAggregate => - super.visit(node, ordinal, parent) - // keys are always forwarded by Over aggregate + // keyAncestors are always forwarded by Over aggregate + visit(node.getInput(0)) case a: DataStreamGroupAggregate => - // get grouping keys + // get grouping keyAncestors val groupKeys = a.getRowType.getFieldNames.asScala.take(a.getGroupings.length) - keys = Some(groupKeys.toArray) + Some(groupKeys.map(e => (e, e)).toList) case w: DataStreamGroupWindowAggregate => - // get grouping keys + // get grouping keyAncestors val groupKeys = w.getRowType.getFieldNames.asScala.take(w.getGroupings.length).toArray // get window start and end time val windowStartEnd = w.getWindowProperties.map(_.name) // we have only a unique key if at least one window property is selected if (windowStartEnd.nonEmpty) { - keys = Some(groupKeys ++ windowStartEnd) + Some((groupKeys ++ windowStartEnd).map(e => (e, e)).toList) + } else { + None + } + + case j: DataStreamJoin => + val leftKeyAncestors = visit(j.getLeft) + val rightKeyAncestors = visit(j.getRight) + if (!leftKeyAncestors.isDefined || !rightKeyAncestors.isDefined) { + None + } else { + // both left and right contain keys + val leftJoinKeys = --- End diff -- Is easier to compute with: ``` val leftFieldNames = j.getLeft.getRowType.getFieldNames val leftJoinKeys: Seq[String] = j.getJoinInfo.leftKeys.asScala.map(leftFieldNames.get(_)) ``` > 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)