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

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

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

    https://github.com/apache/flink/pull/3269#discussion_r103960833
  
    --- Diff: 
flink-libraries/flink-table/src/test/scala/org/apache/flink/table/plan/rules/util/RexProgramProjectExtractorTest.scala
 ---
    @@ -55,6 +56,34 @@ class RexProgramProjectExtractorTest {
       }
     
       @Test
    +  def testExtractRefNestedInputFields(): Unit = {
    +    val rexProgram = buildRexProgramWithNesting()
    +    val usedFields = extractRefInputFields(rexProgram)
    +    val usedNestedFields = extractRefNestedInputFields(rexProgram, 
usedFields)
    +    val expected = Array[Array[String]](Array("amount"), Array("*"))
    +    assertThat(usedNestedFields, is(expected))
    +  }
    +
    +  @Test
    +  def testExtractRefNestedInputFieldsWithNoNesting(): Unit = {
    +    val rexProgram = buildRexProgram()
    +    val usedFields = extractRefInputFields(rexProgram)
    +    val usedNestedFields = extractRefNestedInputFields(rexProgram, 
usedFields)
    +    val expected = Array[Array[String]](Array("*"), Array("*"), Array("*"))
    +    assertThat(usedNestedFields, is(expected))
    +  }
    +
    +  @Test
    +  def testExtractDeepRefNestedInputFields(): Unit = {
    +    val rexProgram = buildRexProgramWithDeepNesting()
    +    val usedFields = extractRefInputFields(rexProgram)
    +    val usedNestedFields = extractRefNestedInputFields(rexProgram, 
usedFields)
    +    val expected = Array[Array[String]](Array("amount"), 
Array("passport.status"))
    --- End diff --
    
    Another test would be to reference the nested attribute in a call, for 
example something like `payments.amount * 10`.


> Add NestedFieldsProjectableTableSource interface
> ------------------------------------------------
>
>                 Key: FLINK-5698
>                 URL: https://issues.apache.org/jira/browse/FLINK-5698
>             Project: Flink
>          Issue Type: New Feature
>          Components: Table API & SQL
>            Reporter: Anton Solovev
>            Assignee: Anton Solovev
>
> Add a NestedFieldsProjectableTableSource interface for some TableSource 
> implementation that support nesting projection push-down.
> The interface could look as follows
> {code}
> def trait NestedFieldsProjectableTableSource {
>   def projectNestedFields(fields: Array[String]): 
> NestedFieldsProjectableTableSource[T]
> }
> {code}
> This interface works together with ProjectableTableSource 



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to