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 88DDC200C72 for ; Fri, 12 May 2017 18:40:21 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 8757B160BB8; Fri, 12 May 2017 16:40:21 +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 CFE63160BC8 for ; Fri, 12 May 2017 18:40:20 +0200 (CEST) Received: (qmail 37019 invoked by uid 500); 12 May 2017 16:40:20 -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 37009 invoked by uid 99); 12 May 2017 16:40:20 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 May 2017 16:40:20 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 906D11A7B0D for ; Fri, 12 May 2017 16:40:19 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.02 X-Spam-Level: X-Spam-Status: No, score=-4.02 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.001, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id GCL9wGMY1QR2 for ; Fri, 12 May 2017 16:40:17 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id 247FD5FDDC for ; Fri, 12 May 2017 16:40:16 +0000 (UTC) Received: (qmail 36520 invoked by uid 99); 12 May 2017 16:40:16 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 12 May 2017 16:40:16 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 353CEE038B; Fri, 12 May 2017 16:40:16 +0000 (UTC) From: fhueske To: issues@flink.incubator.apache.org Reply-To: issues@flink.incubator.apache.org References: In-Reply-To: Subject: [GitHub] flink pull request #3715: [FLINK-6232][Table&SQL]Support proctime inner equi... Content-Type: text/plain Message-Id: <20170512164016.353CEE038B@git1-us-west.apache.org> Date: Fri, 12 May 2017 16:40:16 +0000 (UTC) archived-at: Fri, 12 May 2017 16:40:21 -0000 Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3715#discussion_r116268688 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/datastream/DataStreamJoin.scala --- @@ -0,0 +1,180 @@ +/* + * 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.JoinRelType +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter} +import org.apache.flink.api.common.functions.RichFilterFunction +import org.apache.flink.api.java.functions.NullByteKeySelector +import org.apache.flink.streaming.api.datastream.DataStream +import org.apache.flink.table.api.{StreamTableEnvironment, TableException} +import org.apache.flink.table.calcite.FlinkTypeFactory +import org.apache.flink.table.plan.nodes.CommonJoin +import org.apache.flink.table.plan.nodes.logical.FlinkLogicalJoin +import org.apache.flink.table.plan.schema.RowSchema +import org.apache.flink.table.runtime.join.{JoinUtil, ProcTimeInnerJoin} +import org.apache.flink.table.runtime.types.{CRow, CRowTypeInfo} +import org.apache.flink.types.Row + +/** + * Flink RelNode which matches along with JoinOperator and its related operations. + */ +class DataStreamJoin( + cluster: RelOptCluster, + traitSet: RelTraitSet, + leftNode: RelNode, + rightNode: RelNode, + joinNode: FlinkLogicalJoin, + leftSchema: RowSchema, + schema: RowSchema, + 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 DataStreamJoin( + cluster, + traitSet, + inputs.get(0), + inputs.get(1), + joinNode, + leftSchema, + schema, + ruleDescription) + } + + override def toString: String = { + + s"${joinTypeToString(joinNode.getJoinType)}" + + s"(condition: (${joinConditionToString(schema.logicalType, + joinNode.getCondition, getExpressionString)}), " + + s"select: (${joinSelectionToString(schema.logicalType)}))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + super.explainTerms(pw) + .item("condition", joinConditionToString(schema.logicalType, + joinNode.getCondition, getExpressionString)) + .item("select", joinSelectionToString(schema.logicalType)) + .item("joinType", joinTypeToString(joinNode.getJoinType)) + } + + override def translateToPlan(tableEnv: StreamTableEnvironment): DataStream[CRow] = { + + val config = tableEnv.getConfig + + // get the equality keys and other condition + val (leftKeys, rightKeys, otherCondition) = + JoinUtil.analyzeJoinCondition(joinNode, getExpressionString) + + if (left.isInstanceOf[StreamTableSourceScan] --- End diff -- @hongyuhong, why do we need this condition? --- 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. ---