[ https://issues.apache.org/jira/browse/FLINK-4541?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15693124#comment-15693124 ]
ASF GitHub Bot commented on FLINK-4541: --------------------------------------- Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/2811#discussion_r89482516 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/api/table/plan/nodes/dataset/DataSetSingleRowCross.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.api.table.plan.nodes.dataset + +import org.apache.calcite.plan._ +import org.apache.calcite.rel.`type`.RelDataType +import org.apache.calcite.rel.metadata.RelMetadataQuery +import org.apache.calcite.rel.{BiRel, RelNode, RelWriter} +import org.apache.flink.api.common.functions.{MapFunction, RichMapFunction} +import org.apache.flink.api.common.typeinfo.TypeInformation +import org.apache.flink.api.java.DataSet +import org.apache.flink.api.table.codegen.CodeGenerator +import org.apache.flink.api.table.runtime.RichMapRunner +import org.apache.flink.api.table.typeutils.TypeConverter.determineReturnType +import org.apache.flink.api.table.{BatchTableEnvironment, TableConfig} + +import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ + +/** + * Flink RelNode which matches along with CrossOperator. + */ +class DataSetSingleRowCross( + cluster: RelOptCluster, + traitSet: RelTraitSet, + leftNode: RelNode, + rightNode: RelNode, + leftIsSingle: Boolean, + rowRelDataType: RelDataType, + joinRowType: RelDataType, + ruleDescription: String) + extends BiRel(cluster, traitSet, leftNode, rightNode) + with DataSetRel { + + override def deriveRowType() = rowRelDataType + + override def copy(traitSet: RelTraitSet, inputs: java.util.List[RelNode]): RelNode = { + new DataSetSingleRowCross( + cluster, + traitSet, + inputs.get(0), + inputs.get(1), + leftIsSingle, + getRowType, + joinRowType, + ruleDescription) + } + + override def toString: String = { + s"$joinTypeToString(where: ($joinConditionToString), join: ($joinSelectionToString))" + } + + override def explainTerms(pw: RelWriter): RelWriter = { + super.explainTerms(pw) + .item("where", joinConditionToString) + .item("join", joinSelectionToString) + .item("joinType", joinTypeToString) + } + + override def computeSelfCost (planner: RelOptPlanner, metadata: RelMetadataQuery): RelOptCost = { + val children = this.getInputs + children.foldLeft(planner.getCostFactory.makeZeroCost()) { (cost, child) => + val rowCnt = metadata.getRowCount(child) + val rowSize = this.estimateRowSize(child.getRowType) + cost.plus(planner.getCostFactory.makeCost(rowCnt, rowCnt, rowCnt * rowSize)) + } + } + + override def translateToPlan( + tableEnv: BatchTableEnvironment, + expectedType: Option[TypeInformation[Any]]): DataSet[Any] = { + + val leftDataSet = left.asInstanceOf[DataSetRel].translateToPlan(tableEnv) + val rightDataSet = right.asInstanceOf[DataSetRel].translateToPlan(tableEnv) + val broadcastSetName = "joinSet" + val mapSideJoin = generateMapFunction( + tableEnv.getConfig, + leftDataSet.getType, + rightDataSet.getType, + leftIsSingle, + broadcastSetName, + expectedType) + + val (multiRowDataSet, singleRowDataSet) = + if (leftIsSingle) { + (rightDataSet, leftDataSet) + } else { + (leftDataSet, rightDataSet) + } + + multiRowDataSet + .map(mapSideJoin) + .withBroadcastSet(singleRowDataSet, broadcastSetName) + .name(getMapOperatorName) + .asInstanceOf[DataSet[Any]] + } + + private def generateMapFunction( --- End diff -- Actually, I think it is easier and requires less additional code to generate a regular `JoinFunction` as done in the `DataSetJoin` and call it from the wrapping `RichMapFunction` with the input value from `map()` and the broadcasted value. With this we can move the broadcast and `RichFunction` logic completely to the wrapper and do not need any changes to the code generation. > Support for SQL NOT IN operator > ------------------------------- > > Key: FLINK-4541 > URL: https://issues.apache.org/jira/browse/FLINK-4541 > Project: Flink > Issue Type: Improvement > Components: Table API & SQL > Reporter: Timo Walther > Assignee: Alexander Shoshin > > This should work: > {code} > def main(args: Array[String]): Unit = { > // set up execution environment > val env = ExecutionEnvironment.getExecutionEnvironment > val tEnv = TableEnvironment.getTableEnvironment(env) > val input = env.fromElements(WC("hello", 1), WC("hello", 1), WC("ciao", > 1)) > // register the DataSet as table "WordCount" > tEnv.registerDataSet("WordCount", input, 'word, 'frequency) > tEnv.registerTable("WordCount2", tEnv.fromDataSet(input, 'word, > 'frequency).select('word).filter('word !== "hello")) > // run a SQL query on the Table and retrieve the result as a new Table > val table = tEnv.sql("SELECT word, SUM(frequency) FROM WordCount WHERE > word NOT IN (SELECT word FROM WordCount2) GROUP BY word") > table.toDataSet[WC].print() > } > {code} -- This message was sent by Atlassian JIRA (v6.3.4#6332)