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

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

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

    https://github.com/apache/flink/pull/3040#discussion_r95975700
  
    --- Diff: 
flink-libraries/flink-table/src/main/scala/org/apache/flink/table/plan/nodes/dataset/forwarding/FieldForwardingUtils.scala
 ---
    @@ -0,0 +1,115 @@
    +/*
    + * 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.dataset.forwarding
    +
    +import org.apache.calcite.rex.RexProgram
    +import org.apache.flink.api.common.typeinfo.{BasicTypeInfo, 
TypeInformation}
    +import org.apache.flink.api.common.typeutils.CompositeType
    +import org.apache.flink.table.api.TableException
    +
    +import scala.collection.JavaConversions._
    +
    +object FieldForwardingUtils {
    +
    +  def compositeTypeField = (fields: Seq[String]) => (v: Int) => fields(v)
    +
    +  private def throwMissedWrapperException(wrapperCustomCase: 
TypeInformation[_]) = {
    +    throw new TableException(s"Implementation for $wrapperCustomCase index 
wrapper is missing.")
    +  }
    +
    +  /**
    +    * Wrapper for {@link getForwardedFields}
    +    * @param inputType
    +    * @param outputType
    +    * @param forwardIndices
    +    * @param wrapperCustomCase
    +    * @param calcProgram
    +    * @return
    +    */
    +  def getForwardedInput(
    +      inputType: TypeInformation[_],
    +      outputType: TypeInformation[_],
    +      forwardIndices: Seq[Int],
    +      wrapperCustomCase: TypeInformation[_] => (Int) => String = 
throwMissedWrapperException,
    +      calcProgram: Option[RexProgram] = None) = {
    +
    +    getForwardedFields(inputType,
    +      outputType,
    +      forwardIndices.zip(forwardIndices),
    +      wrapperCustomCase,
    +      calcProgram)
    +  }
    +
    +  /**
    +    * Wraps provided indices with proper names, e.g. _1 for tuple, f0 for 
row, fieldName for POJO.
    +    * @param inputType
    +    * @param outputType
    +    * @param forwardIndices - tuple of input-output indices of a forwarded 
field
    +    * @param wrapperCustomCase - used for  figuring out proper type in 
specific cases,
    +    *                          e.g. {@see DataSetSingleRowJoin}
    +    * @param calcProgram - used for  figuring out proper type in specific 
cases,
    +    *                    e.g. {@see DataSetCalc}
    +    * @return - string with forwarded fields mapped from input to output
    +    */
    +  def getForwardedFields(
    +      inputType: TypeInformation[_],
    +      outputType: TypeInformation[_],
    +      forwardIndices: Seq[(Int, Int)],
    +      wrapperCustomCase: TypeInformation[_] => (Int) => String = 
throwMissedWrapperException,
    +      calcProgram: Option[RexProgram] = None): String = {
    +
    +    def chooseWrapper(typeInformation: TypeInformation[_]): (Int) => 
String = {
    +      typeInformation match {
    +        case composite: CompositeType[_] =>
    +          // POJOs' fields are sorted, so we can not access them by their 
positional index.
    +          // So we collect field names from
    +          // outputRowType. For all other types we get field names from 
inputDS.
    +          val typeFieldList = composite.getFieldNames
    +          var fieldWrapper: (Int) => String = 
compositeTypeField(typeFieldList)
    +          if (calcProgram.isDefined) {
    +            val projectFieldList = 
calcProgram.get.getOutputRowType.getFieldNames
    --- End diff --
    
    I think this would break the order of the fields.


> Add forward field annotations to DataSet operators generated by the Table API
> -----------------------------------------------------------------------------
>
>                 Key: FLINK-3850
>                 URL: https://issues.apache.org/jira/browse/FLINK-3850
>             Project: Flink
>          Issue Type: Improvement
>          Components: Table API & SQL
>            Reporter: Fabian Hueske
>            Assignee: Nikolay Vasilishin
>
> The DataSet API features semantic annotations [1] to hint the optimizer which 
> input fields an operator copies. This information is valuable for the 
> optimizer because it can infer that certain physical properties such as 
> partitioning or sorting are not destroyed by user functions and thus generate 
> more efficient execution plans.
> The Table API is built on top of the DataSet API and generates DataSet 
> programs and code for user-defined functions. Hence, it knows exactly which 
> fields are modified and which not. We should use this information to 
> automatically generate forward field annotations and attach them to the 
> operators. This can help to significantly improve the performance of certain 
> jobs.
> [1] 
> https://ci.apache.org/projects/flink/flink-docs-release-1.0/apis/batch/index.html#semantic-annotations



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to