You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/14 10:40:06 UTC

[GitHub] [flink] fsk119 opened a new pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

fsk119 opened a new pull request #13631:
URL: https://github.com/apache/flink/pull/13631


   …own in planner
   
   <!--
   *Thank you very much for contributing to Apache Flink - we are happy that you want to help us improve Flink. To help the community review your contribution in the best possible way, please go through the checklist below, which will get the contribution into a shape in which it can be best reviewed.*
   
   *Please understand that we do not do this to make contributions to Flink a hassle. In order to uphold a high standard of quality for code contributions, while at the same time managing a large number of contributions, we need contributors to prepare the contributions well, and give reviewers enough contextual information for the review. Please also understand that contributions that do not follow this guide will take longer to review and thus typically be picked up with lower priority by the community.*
   
   ## Contribution Checklist
   
     - Make sure that the pull request corresponds to a [JIRA issue](https://issues.apache.org/jira/projects/FLINK/issues). Exceptions are made for typos in JavaDoc or documentation files, which need no JIRA issue.
     
     - Name the pull request in the form "[FLINK-XXXX] [component] Title of the pull request", where *FLINK-XXXX* should be replaced by the actual issue number. Skip *component* if you are unsure about which is the best component.
     Typo fixes that have no associated JIRA issue should be named following this pattern: `[hotfix] [docs] Fix typo in event time introduction` or `[hotfix] [javadocs] Expand JavaDoc for PuncuatedWatermarkGenerator`.
   
     - Fill out the template below to describe the changes contributed by the pull request. That will give reviewers the context they need to do the review.
     
     - Make sure that the change passes the automated tests, i.e., `mvn clean verify` passes. You can set up Azure Pipelines CI to do that following [this guide](https://cwiki.apache.org/confluence/display/FLINK/Azure+Pipelines#AzurePipelines-Tutorial:SettingupAzurePipelinesforaforkoftheFlinkrepository).
   
     - Each pull request should address only one issue, not mix up code from multiple issues.
     
     - Each commit in the pull request has a meaningful commit message (including the JIRA id)
   
     - Once all items of the checklist are addressed, remove the above text and this checklist, leaving only the filled out template below.
   
   
   **(The sections below can be removed for hotfixes of typos)**
   -->
   
   ## What is the purpose of the change
   
   *Support to push nested projection into `TableSourceScan`. But it may cause name conflicts in some situaion. For example, we create the table with ddl
   `CREATE TABLE NestedTable (
        nest1 ROW<a INT>,
        nest2 ROW<a INT>
   )`
   and with query
   `SELECT nest1.a, nest2.a from NestedTable`
   and we will get 2 `a` in the new schema when pushing projection. 
    Here we use `'_'` to concatenate the names of all levels as the name of the nested fields. In this example, we will get fields `nest1_a`, `nest2_a` in the new schema.*
   
   
   ## Brief change log
   
     - *add `RexNodeReWriter#rewriteNestedProjectionWithNewFieldInput` that will modify the projection with the mapping*
     - *enable rule `PushProjectionIntoTableSourceScanRule` to push nested projection*
   
   
   ## Verifying this change
   
   This change added tests and can be verified as follows:
   
     - *add tests for `RexNodeReWriter#rewriteNestedProjectionWithNewFieldInput`*
     - *add all kinds of tests for rule*
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / **no** / don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (**yes** / no)
     - If yes, how is the feature documented? (not applicable / docs / **JavaDocs** / not documented)
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] fsk119 commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
fsk119 commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r506037128



##########
File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java
##########
@@ -107,4 +96,13 @@ private void testNestedProject(boolean nestedProjectionSupported) {
 		util().verifyPlan(sqlQuery);
 	}
 
+	@Test
+	public void testComplicatedNestedProject() {
+		String sqlQuery = "SELECT id," +
+				"    deepNested.nested1.name AS nestedName,\n" +
+				"    deepNested.nested2 AS nested2,\n" +
+				"    deepNested.nested2.num AS nestedNum\n" +
+				"FROM NestedTable";
+		util().verifyPlan(sqlQuery);

Review comment:
       Add test with calculation in query.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 0707f7489676ee1a7e709479e47c6a3da5dd0e8b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841) 
   * bb84157d91d0f564a199c6af00209db8f51a7dda UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r514907339



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -207,27 +208,35 @@ private DataType applyUpdateMetadataAndGetNewDataType(
 				.filter(usedMetadataKeysUnordered::contains)
 				.collect(Collectors.toList());
 
-		final List<List<Integer>> projectedMetadataFields = usedMetadataKeys
-				.stream()
-				.map(metadataKeys::indexOf)
-				.map(i -> {
-					fieldCoordinatesToOrder.put(physicalFieldCount + i, Collections.singletonMap(Collections.singletonList("*"), fieldCoordinatesToOrder.size()));
-					return Collections.singletonList(physicalFieldCount + i); })
-				.collect(Collectors.toList());
+		final List<List<Integer>> projectedMetadataFields = new ArrayList<>(usedMetadataKeys.size());

Review comment:
       This line is unnecessary, use `usedFieldsCoordinates.add(Collections.singletonList(physicalFieldCount + index))` at line 217




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] fsk119 commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
fsk119 commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r514887936



##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -109,73 +112,67 @@ public void onMatch(RelOptRuleCall call) {
 			usedFields = refFields;
 		}
 		// if no fields can be projected, we keep the original plan.
-		if (usedFields.length == fieldCount) {
+		if (!supportsNestedProjection && usedFields.length == fieldCount) {
 			return;
 		}
 
-		final List<String> projectedFieldNames = IntStream.of(usedFields)
-			.mapToObj(fieldNames::get)
-			.collect(Collectors.toList());
-
 		final TableSchema oldSchema = oldTableSourceTable.catalogTable().getSchema();
 		final DynamicTableSource oldSource = oldTableSourceTable.tableSource();
 		final List<String> metadataKeys = DynamicSourceUtils.createRequiredMetadataKeys(oldSchema, oldSource);
 		final int physicalFieldCount = fieldCount - metadataKeys.size();
 		final DynamicTableSource newSource = oldSource.copy();
 
-		// remove metadata columns from the projection push down and store it in a separate list
-		// the projection push down itself happens purely on physical columns
-		final int[] usedPhysicalFields;
-		final List<String> usedMetadataKeys;
-		if (newSource instanceof SupportsReadingMetadata) {
-			usedPhysicalFields = IntStream.of(usedFields)
-				// select only physical columns
-				.filter(i -> i < physicalFieldCount)
-				.toArray();
-			final List<String> usedMetadataKeysUnordered = IntStream.of(usedFields)
-				// select only metadata columns
-				.filter(i -> i >= physicalFieldCount)
-				// map the indices to keys
-				.mapToObj(i -> metadataKeys.get(fieldCount - i - 1))
-				.collect(Collectors.toList());
-			// order the keys according to the source's declaration
-			usedMetadataKeys = metadataKeys
-				.stream()
-				.filter(usedMetadataKeysUnordered::contains)
-				.collect(Collectors.toList());
+		final List<List<Integer>> usedFieldsCoordinates = new ArrayList<>();
+		final Map<Integer, Map<List<String>, Integer>> fieldCoordinatesToOrder = new HashMap<>();
+
+		if (supportsNestedProjection) {
+			getCoordinatesAndMappingOfPhysicalColumnWithNestedProjection(
+					project, oldSchema, usedFields, physicalFieldCount, usedFieldsCoordinates, fieldCoordinatesToOrder);
 		} else {
-			usedPhysicalFields = usedFields;
-			usedMetadataKeys = Collections.emptyList();
+			for (int usedField : usedFields) {
+				// filter metadata columns

Review comment:
       add TODO.
   Here we only project the top level of the fields.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r506915187



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala
##########
@@ -256,7 +256,15 @@ class RefFieldAccessorVisitor(usedFields: Array[Int]) extends RexVisitorImpl[Uni
     if (right.length < left.length) {
       false
     } else {
-      right.take(left.length).equals(left)
+      right.take(left.length).zip(left).foldLeft(true) {
+        (ans, fields) => {
+          if (ans) {
+            fields._1.equals(fields._2)

Review comment:
       It's better we does not use `_1`, `_2`, which makes the code hard to read. we can use use `case` match with meaningful name, just like:
   ```
   right.take(left.length).zip(left).foldLeft(true) {
          case (ans, (rName, lName)) => {
             if (ans) {
               lName.equals(rName)
             } else {
               false
             }
           }
         }
   ```

##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/utils/TableSchemaUtils.java
##########
@@ -75,10 +79,46 @@ public static TableSchema projectSchema(TableSchema tableSchema, int[][] project
 		checkArgument(containsPhysicalColumnsOnly(tableSchema), "Projection is only supported for physical columns.");
 		TableSchema.Builder schemaBuilder = TableSchema.builder();
 		List<TableColumn> tableColumns = tableSchema.getTableColumns();
+		Map<String, String> nameDomain = new HashMap<>();
+		String exceptionTemplate = "Get name conflicts for origin fields %s and %s with new name `%s`. " +
+				"When pushing projection into scan, we will concatenate top level names with delimiter '_'. " +
+				"Please rename the origin field names when creating table.";
+		String originFullyQualifiedName;
+		String newName;
 		for (int[] fieldPath : projectedFields) {
-			checkArgument(fieldPath.length == 1, "Nested projection push down is not supported yet.");
-			TableColumn column = tableColumns.get(fieldPath[0]);
-			schemaBuilder.field(column.getName(), column.getType());
+			if (fieldPath.length == 1) {
+				TableColumn column = tableColumns.get(fieldPath[0]);
+				newName = column.getName();
+				originFullyQualifiedName = String.format("`%s`", column.getName());
+				if (nameDomain.containsKey(column.getName())) {
+					throw new TableException(

Review comment:
       how about resolve the conflicts through adding postfix ?

##########
File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java
##########
@@ -70,41 +69,40 @@ public void setup() {
 						" 'bounded' = 'true'\n" +
 						")";
 		util().tableEnv().executeSql(ddl2);
-	}
-
-	@Override
-	public void testNestedProject() {
-		expectedException().expect(TableException.class);
-		expectedException().expectMessage("Nested projection push down is unsupported now.");
-		testNestedProject(true);
-	}
 
-	@Test
-	public void testNestedProjectDisabled() {
-		testNestedProject(false);
-	}
-
-	private void testNestedProject(boolean nestedProjectionSupported) {
-		String ddl =
+		String ddl3 =
 				"CREATE TABLE NestedTable (\n" +
 						"  id int,\n" +
-						"  deepNested row<nested1 row<name string, `value` int>, nested2 row<num int, flag boolean>>,\n" +
-						"  nested row<name string, `value` int>,\n" +
+						"  deepNested row<nested1 row<name string, `value` int>, `nested2.` row<num int, flag boolean>>,\n" +
+						"  nested row<name string, `value.` int>,\n" +
 						"  name string\n" +
 						") WITH (\n" +
 						" 'connector' = 'values',\n" +
-						" 'nested-projection-supported' = '" + nestedProjectionSupported + "',\n" +
+						" 'nested-projection-supported' = 'true'," +
 						"  'bounded' = 'true'\n" +
 						")";
-		util().tableEnv().executeSql(ddl);
+		util().tableEnv().executeSql(ddl3);
+	}
 
+	@Override
+	@Test
+	public void testNestedProject() {
 		String sqlQuery = "SELECT id,\n" +
 				"    deepNested.nested1.name AS nestedName,\n" +
-				"    nested.`value` AS nestedValue,\n" +
-				"    deepNested.nested2.flag AS nestedFlag,\n" +
-				"    deepNested.nested2.num AS nestedNum\n" +
+				"    nested.`value.` AS nestedValue,\n" +
+				"    deepNested.`nested2.`.flag AS nestedFlag,\n" +

Review comment:
       how about keep the original test, and add a new test case the verify the case which field name contain dot




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 0707f7489676ee1a7e709479e47c6a3da5dd0e8b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841) 
   * bb84157d91d0f564a199c6af00209db8f51a7dda Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897) 
   * fd8961181c3838b57b78d28c22029d581031763e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * da9bc2d9410e070611621051bcce5461443335a5 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270) 
   * 1697a20356799dd5e2f8863110de1f4626664f12 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 1697a20356799dd5e2f8863110de1f4626664f12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 4b3b9898ceeebc31777e2aa96d04b445cf578d87 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720) 
   * d1b917bb33f837bf04e8351affe7e108911e79b4 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * d1b917bb33f837bf04e8351affe7e108911e79b4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * bb84157d91d0f564a199c6af00209db8f51a7dda Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897) 
   * fd8961181c3838b57b78d28c22029d581031763e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907) 
   * 4bc536663a8a89fee69b58494c76b8e6f5e98c25 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r514204560



##########
File path: flink-table/flink-table-common/src/main/java/org/apache/flink/table/types/utils/DataTypeUtils.java
##########
@@ -74,42 +76,46 @@
 	 *
 	 * <p>Note: Index paths allow for arbitrary deep nesting. For example, {@code [[0, 2, 1], ...]}
 	 * specifies to include the 2nd field of the 3rd field of the 1st field in the top-level row.
+	 * Sometimes, it may get name conflicts when extract fields from the row field. Considering the
+	 * the path is unique to extract fields, it makes sense to use the path to the fields with
+	 * delimiter `_` as the new name of the field. For example, the new name of the field `b` in
+	 * the row `a` is `a_b` rather than `b`. But it may still gets name conflicts in some situation,
+	 * such as the field `a_b` in the top level schema. In such situation, it will use the postfix
+	 * in the format '$%d' to resolve the name conflicts.
 	 */
 	public static DataType projectRow(DataType dataType, int[][] indexPaths) {
 		final List<RowField> updatedFields = new ArrayList<>();
 		final List<DataType> updatedChildren = new ArrayList<>();
+		Set<String> nameDomain = new HashSet<>();
+		int duplicateCount = 0;
 		for (int[] indexPath : indexPaths) {
-			updatedFields.add(selectChild(dataType.getLogicalType(), indexPath, 0));
-			updatedChildren.add(selectChild(dataType, indexPath, 0));
+			DataType fieldType = dataType.getChildren().get(indexPath[0]);
+			LogicalType fieldLogicalType = fieldType.getLogicalType();
+			StringBuilder builder =
+					new StringBuilder(((RowType) dataType.getLogicalType()).getFieldNames().get(indexPath[0]));
+			for (int index = 1; index < indexPath.length; index++) {
+				Preconditions.checkArgument(
+						hasRoot(fieldLogicalType, LogicalTypeRoot.ROW),
+						"Row data type expected.");
+				RowType rowtype = ((RowType) fieldLogicalType);
+				builder.append("_").append(rowtype.getFieldNames().get(indexPath[index]));
+				fieldLogicalType = rowtype.getFields().get(indexPath[index]).getType();
+				fieldType = fieldType.getChildren().get(indexPath[index]);
+			}
+			String path = builder.toString();
+			while (nameDomain.contains(path)) {
+				path = builder.append("$").append(duplicateCount++).toString();

Review comment:
       nit: how about adding  `_` before "$"

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -109,73 +112,67 @@ public void onMatch(RelOptRuleCall call) {
 			usedFields = refFields;
 		}
 		// if no fields can be projected, we keep the original plan.
-		if (usedFields.length == fieldCount) {
+		if (!supportsNestedProjection && usedFields.length == fieldCount) {
 			return;
 		}
 
-		final List<String> projectedFieldNames = IntStream.of(usedFields)
-			.mapToObj(fieldNames::get)
-			.collect(Collectors.toList());
-
 		final TableSchema oldSchema = oldTableSourceTable.catalogTable().getSchema();
 		final DynamicTableSource oldSource = oldTableSourceTable.tableSource();
 		final List<String> metadataKeys = DynamicSourceUtils.createRequiredMetadataKeys(oldSchema, oldSource);
 		final int physicalFieldCount = fieldCount - metadataKeys.size();
 		final DynamicTableSource newSource = oldSource.copy();
 
-		// remove metadata columns from the projection push down and store it in a separate list
-		// the projection push down itself happens purely on physical columns
-		final int[] usedPhysicalFields;
-		final List<String> usedMetadataKeys;
-		if (newSource instanceof SupportsReadingMetadata) {
-			usedPhysicalFields = IntStream.of(usedFields)
-				// select only physical columns
-				.filter(i -> i < physicalFieldCount)
-				.toArray();
-			final List<String> usedMetadataKeysUnordered = IntStream.of(usedFields)
-				// select only metadata columns
-				.filter(i -> i >= physicalFieldCount)
-				// map the indices to keys
-				.mapToObj(i -> metadataKeys.get(fieldCount - i - 1))
-				.collect(Collectors.toList());
-			// order the keys according to the source's declaration
-			usedMetadataKeys = metadataKeys
-				.stream()
-				.filter(usedMetadataKeysUnordered::contains)
-				.collect(Collectors.toList());
+		final List<List<Integer>> usedFieldsCoordinates = new ArrayList<>();
+		final Map<Integer, Map<List<String>, Integer>> fieldCoordinatesToOrder = new HashMap<>();
+
+		if (supportsNestedProjection) {
+			getCoordinatesAndMappingOfPhysicalColumnWithNestedProjection(
+					project, oldSchema, usedFields, physicalFieldCount, usedFieldsCoordinates, fieldCoordinatesToOrder);
 		} else {
-			usedPhysicalFields = usedFields;
-			usedMetadataKeys = Collections.emptyList();
+			for (int usedField : usedFields) {
+				// filter metadata columns

Review comment:
       what if the metadata columns have nested fields ?

##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/runtime/batch/sql/TableSourceITCase.scala
##########
@@ -66,6 +66,26 @@ class TableSourceITCase extends BatchTestBase {
          |  'bounded' = 'true'
          |)
          |""".stripMargin)
+    val nestedTableDataId = TestValuesTableFactory.registerData(TestData.deepNestedRow)

Review comment:
       remove unused imports

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -190,35 +187,82 @@ public void onMatch(RelOptRuleCall call) {
 		}
 	}
 
-	private void applyUpdatedMetadata(
-			DynamicTableSource oldSource,
-			TableSchema oldSchema,
+	private DataType applyUpdateMetadataAndGetNewDataType(
 			DynamicTableSource newSource,
+			DataType producedDataType,
 			List<String> metadataKeys,
-			List<String> usedMetadataKeys,
+			int[] usedFields,
 			int physicalFieldCount,
-			int[][] projectedPhysicalFields) {
-		if (newSource instanceof SupportsReadingMetadata) {
-			final DataType producedDataType = TypeConversions.fromLogicalToDataType(
-				DynamicSourceUtils.createProducedType(oldSchema, oldSource));
+			List<List<Integer>> usedFieldsCoordinates,
+			Map<Integer, Map<List<String>, Integer>> fieldCoordinatesToOrder) {
+		final List<String> usedMetadataKeysUnordered = IntStream.of(usedFields)
+				// select only metadata columns
+				.filter(i -> i >= physicalFieldCount)
+				// map the indices to keys
+				.mapToObj(i -> metadataKeys.get(i - physicalFieldCount))
+				.collect(Collectors.toList());
+		// order the keys according to the source's declaration
+		final List<String> usedMetadataKeys = metadataKeys
+				.stream()
+				.filter(usedMetadataKeysUnordered::contains)
+				.collect(Collectors.toList());
 
-			final int[][] projectedMetadataFields = usedMetadataKeys
+		final List<List<Integer>> projectedMetadataFields = usedMetadataKeys
 				.stream()
 				.map(metadataKeys::indexOf)
-				.map(i -> new int[]{ physicalFieldCount + i })
-				.toArray(int[][]::new);
+				.map(i -> {
+					fieldCoordinatesToOrder.put(physicalFieldCount + i, Collections.singletonMap(Collections.singletonList("*"), fieldCoordinatesToOrder.size()));
+					return Collections.singletonList(physicalFieldCount + i); })
+				.collect(Collectors.toList());
+		usedFieldsCoordinates.addAll(projectedMetadataFields);

Review comment:
       use `for` to make it clearer

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -190,35 +187,82 @@ public void onMatch(RelOptRuleCall call) {
 		}
 	}
 
-	private void applyUpdatedMetadata(
-			DynamicTableSource oldSource,
-			TableSchema oldSchema,
+	private DataType applyUpdateMetadataAndGetNewDataType(
 			DynamicTableSource newSource,
+			DataType producedDataType,
 			List<String> metadataKeys,
-			List<String> usedMetadataKeys,
+			int[] usedFields,
 			int physicalFieldCount,
-			int[][] projectedPhysicalFields) {
-		if (newSource instanceof SupportsReadingMetadata) {
-			final DataType producedDataType = TypeConversions.fromLogicalToDataType(
-				DynamicSourceUtils.createProducedType(oldSchema, oldSource));
+			List<List<Integer>> usedFieldsCoordinates,
+			Map<Integer, Map<List<String>, Integer>> fieldCoordinatesToOrder) {
+		final List<String> usedMetadataKeysUnordered = IntStream.of(usedFields)
+				// select only metadata columns
+				.filter(i -> i >= physicalFieldCount)
+				// map the indices to keys
+				.mapToObj(i -> metadataKeys.get(i - physicalFieldCount))
+				.collect(Collectors.toList());
+		// order the keys according to the source's declaration
+		final List<String> usedMetadataKeys = metadataKeys
+				.stream()
+				.filter(usedMetadataKeysUnordered::contains)
+				.collect(Collectors.toList());
 
-			final int[][] projectedMetadataFields = usedMetadataKeys
+		final List<List<Integer>> projectedMetadataFields = usedMetadataKeys
 				.stream()
 				.map(metadataKeys::indexOf)
-				.map(i -> new int[]{ physicalFieldCount + i })
-				.toArray(int[][]::new);
+				.map(i -> {
+					fieldCoordinatesToOrder.put(physicalFieldCount + i, Collections.singletonMap(Collections.singletonList("*"), fieldCoordinatesToOrder.size()));
+					return Collections.singletonList(physicalFieldCount + i); })
+				.collect(Collectors.toList());
+		usedFieldsCoordinates.addAll(projectedMetadataFields);
 
-			final int[][] projectedFields = Stream
-				.concat(
-					Stream.of(projectedPhysicalFields),
-					Stream.of(projectedMetadataFields)
-				)
+		int[][] allFields = usedFieldsCoordinates
+				.stream()
+				.map(coordinates -> coordinates.stream().mapToInt(i -> i).toArray())
 				.toArray(int[][]::new);
 
-			// create a new, final data type that includes all projections
-			final DataType newProducedDataType = DataTypeUtils.projectRow(producedDataType, projectedFields);
+		DataType newProducedDataType = DataTypeUtils.projectRow(producedDataType, allFields);
 
-			((SupportsReadingMetadata) newSource).applyReadableMetadata(usedMetadataKeys, newProducedDataType);
+		((SupportsReadingMetadata) newSource).applyReadableMetadata(usedMetadataKeys, newProducedDataType);
+		return newProducedDataType;
+	}
+
+	private void getCoordinatesAndMappingOfPhysicalColumnWithNestedProjection(

Review comment:
       the method name is too long, change to `getExpandedFieldsAndOrderMapping` ? add some comments the explain the arguments

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -74,28 +79,26 @@ public boolean matches(RelOptRuleCall call) {
 		if (tableSourceTable == null || !(tableSourceTable.tableSource() instanceof SupportsProjectionPushDown)) {
 			return false;
 		}
-		SupportsProjectionPushDown pushDownSource = (SupportsProjectionPushDown) tableSourceTable.tableSource();
-		if (pushDownSource.supportsNestedProjection()) {
-			throw new TableException("Nested projection push down is unsupported now. \n" +
-					"Please disable nested projection (SupportsProjectionPushDown#supportsNestedProjection returns false), " +
-					"planner will push down the top-level columns.");
-		} else {
-			return true;
-		}
+		return Arrays.stream(tableSourceTable.extraDigests()).noneMatch(digest -> digest.startsWith("project=["));
 	}
 
 	@Override
 	public void onMatch(RelOptRuleCall call) {
 		final LogicalProject project = call.rel(0);
 		final LogicalTableScan scan = call.rel(1);
 
+		TableSourceTable oldTableSourceTable = scan.getTable().unwrap(TableSourceTable.class);
+
+		final boolean supportsNestedProjection =
+				((SupportsProjectionPushDown) oldTableSourceTable.tableSource()).supportsNestedProjection();
+		final boolean supportsReadingMetaData = oldTableSourceTable.tableSource() instanceof SupportsReadingMetadata;

Review comment:
       nit: It is better to close the position of the defined field to the position in which the field is used.

##########
File path: flink-table/flink-table-planner-blink/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java
##########
@@ -190,35 +187,82 @@ public void onMatch(RelOptRuleCall call) {
 		}
 	}
 
-	private void applyUpdatedMetadata(
-			DynamicTableSource oldSource,
-			TableSchema oldSchema,
+	private DataType applyUpdateMetadataAndGetNewDataType(
 			DynamicTableSource newSource,
+			DataType producedDataType,
 			List<String> metadataKeys,
-			List<String> usedMetadataKeys,
+			int[] usedFields,
 			int physicalFieldCount,
-			int[][] projectedPhysicalFields) {
-		if (newSource instanceof SupportsReadingMetadata) {
-			final DataType producedDataType = TypeConversions.fromLogicalToDataType(
-				DynamicSourceUtils.createProducedType(oldSchema, oldSource));
+			List<List<Integer>> usedFieldsCoordinates,
+			Map<Integer, Map<List<String>, Integer>> fieldCoordinatesToOrder) {
+		final List<String> usedMetadataKeysUnordered = IntStream.of(usedFields)
+				// select only metadata columns
+				.filter(i -> i >= physicalFieldCount)
+				// map the indices to keys
+				.mapToObj(i -> metadataKeys.get(i - physicalFieldCount))
+				.collect(Collectors.toList());
+		// order the keys according to the source's declaration
+		final List<String> usedMetadataKeys = metadataKeys
+				.stream()
+				.filter(usedMetadataKeysUnordered::contains)
+				.collect(Collectors.toList());
 
-			final int[][] projectedMetadataFields = usedMetadataKeys
+		final List<List<Integer>> projectedMetadataFields = usedMetadataKeys
 				.stream()
 				.map(metadataKeys::indexOf)
-				.map(i -> new int[]{ physicalFieldCount + i })
-				.toArray(int[][]::new);
+				.map(i -> {
+					fieldCoordinatesToOrder.put(physicalFieldCount + i, Collections.singletonMap(Collections.singletonList("*"), fieldCoordinatesToOrder.size()));
+					return Collections.singletonList(physicalFieldCount + i); })
+				.collect(Collectors.toList());
+		usedFieldsCoordinates.addAll(projectedMetadataFields);
 
-			final int[][] projectedFields = Stream
-				.concat(
-					Stream.of(projectedPhysicalFields),
-					Stream.of(projectedMetadataFields)
-				)
+		int[][] allFields = usedFieldsCoordinates

Review comment:
       allFields -> projectedFields

##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala
##########
@@ -79,7 +78,7 @@ class RexNodeExtractorTest extends RexNodeTestBase {
     val usedFields = RexNodeExtractor.extractRefInputFields(rexProgram)
     val usedNestedFields = RexNodeExtractor.extractRefNestedInputFields(rexProgram, usedFields)
 
-    val expected = Array(Array("amount"), Array("*"))
+    val expected = Array(Array(util.Arrays.asList("amount")), Array(util.Arrays.asList("*")))

Review comment:
       nit: please delete a unused method `assertPlannerExpressionArrayEquals`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252) 
   * 2fdb44da037e75537996e45b164f2a7968c78c73 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255) 
   * da9bc2d9410e070611621051bcce5461443335a5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] fsk119 commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
fsk119 commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r514888591



##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractorTest.scala
##########
@@ -79,7 +78,7 @@ class RexNodeExtractorTest extends RexNodeTestBase {
     val usedFields = RexNodeExtractor.extractRefInputFields(rexProgram)
     val usedNestedFields = RexNodeExtractor.extractRefNestedInputFields(rexProgram, usedFields)
 
-    val expected = Array(Array("amount"), Array("*"))
+    val expected = Array(Array(util.Arrays.asList("amount")), Array(util.Arrays.asList("*")))

Review comment:
       I has rm the unused method.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 4bc536663a8a89fee69b58494c76b8e6f5e98c25 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8936",
       "triggerID" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 4bc536663a8a89fee69b58494c76b8e6f5e98c25 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883) 
   * 5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8936) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8936",
       "triggerID" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8936) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803) 
   * 46f99482ad7bea8e467130fe5dd92b61b7d3d444 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252) 
   * 2fdb44da037e75537996e45b164f2a7968c78c73 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255) 
   * da9bc2d9410e070611621051bcce5461443335a5 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * da9bc2d9410e070611621051bcce5461443335a5 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270) 
   * 1697a20356799dd5e2f8863110de1f4626664f12 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603) 
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252) 
   * 2fdb44da037e75537996e45b164f2a7968c78c73 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 803f22afa05eac973c3c9e7b69722b4840c8173f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600) 
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252) 
   * 2fdb44da037e75537996e45b164f2a7968c78c73 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 1697a20356799dd5e2f8863110de1f4626664f12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519) 
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 9898b59d09c7483e3e0c034372a4875c46844841 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r514952517



##########
File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java
##########
@@ -70,34 +70,47 @@ public void setup() {
 						" 'bounded' = 'true'\n" +
 						")";
 		util().tableEnv().executeSql(ddl2);
-	}
-
-	@Override
-	public void testNestedProject() {
-		expectedException().expect(TableException.class);
-		expectedException().expectMessage("Nested projection push down is unsupported now.");
-		testNestedProject(true);
-	}
-
-	@Test
-	public void testNestedProjectDisabled() {
-		testNestedProject(false);
-	}
 
-	private void testNestedProject(boolean nestedProjectionSupported) {
-		String ddl =
+		String ddl3 =
 				"CREATE TABLE NestedTable (\n" +
 						"  id int,\n" +
 						"  deepNested row<nested1 row<name string, `value` int>, nested2 row<num int, flag boolean>>,\n" +
 						"  nested row<name string, `value` int>,\n" +
+						"  `deepNestedWith.` row<`.value` int, nested row<name string, `.value` int>>,\n" +
 						"  name string\n" +
 						") WITH (\n" +
 						" 'connector' = 'values',\n" +
-						" 'nested-projection-supported' = '" + nestedProjectionSupported + "',\n" +
-						"  'bounded' = 'true'\n" +
+						" 'nested-projection-supported' = 'true'," +
+						" 'bounded' = 'true'\n" +
+						")";
+		util().tableEnv().executeSql(ddl3);
+
+		String ddl4 =
+				"CREATE TABLE MetadataTable(\n" +
+						"  id int,\n" +
+						"  deepNested row<nested1 row<name string, `value` int>, nested2 row<num int, flag boolean>>,\n" +
+						"  metadata_1 int metadata,\n" +
+						"  metadata_2 string metadata\n" +
+						") WITH (" +
+						" 'connector' = 'values'," +
+						" 'nested-projection-supported' = 'true'," +
+						" 'bounded' = 'true',\n" +
+						" 'readable-metadata' = 'metadata_1:INT, metadata_2:STRING, metadata_3:BIGINT'" +
 						")";
-		util().tableEnv().executeSql(ddl);
+		util().tableEnv().executeSql(ddl4);
+	}
 
+	@Test
+	public void testProjectWithMapType() {
+		String sqlQuery =
+				"SELECT a, d['e']\n" +
+						"FROM MyTable";

Review comment:
       MyTable does not support `nested-projection-supported`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 803f22afa05eac973c3c9e7b69722b4840c8173f UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * fd8961181c3838b57b78d28c22029d581031763e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907) 
   * 4bc536663a8a89fee69b58494c76b8e6f5e98c25 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603) 
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 4b3b9898ceeebc31777e2aa96d04b445cf578d87 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 803f22afa05eac973c3c9e7b69722b4840c8173f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 46f99482ad7bea8e467130fe5dd92b61b7d3d444 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 803f22afa05eac973c3c9e7b69722b4840c8173f Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600) 
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603) 
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 4b3b9898ceeebc31777e2aa96d04b445cf578d87 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 0707f7489676ee1a7e709479e47c6a3da5dd0e8b Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841) 
   * bb84157d91d0f564a199c6af00209db8f51a7dda Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897) 
   * fd8961181c3838b57b78d28c22029d581031763e UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803) 
   * 46f99482ad7bea8e467130fe5dd92b61b7d3d444 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 2fdb44da037e75537996e45b164f2a7968c78c73 Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255) 
   * da9bc2d9410e070611621051bcce5461443335a5 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] fsk119 commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
fsk119 commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r506036775



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala
##########
@@ -268,7 +268,8 @@ class RefFieldAccessorVisitor(usedFields: Array[Int]) extends RexVisitorImpl[Uni
             // access is top-level access => return top-level access
             case _ :: _ if nestedAccess.equals("*") => List("*")
             // previous access is not prefix of this access => add access
-            case head :: _ if !nestedAccess.startsWith(head) =>
+            // it may cause bug without "." as tail if we have references a.b and a.bb
+            case head :: _ if !nestedAccess.startsWith(head + ".") =>

Review comment:
       I have use qualified name list to determine whether the element has been added into new schema.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r505333391



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeExtractor.scala
##########
@@ -268,7 +268,8 @@ class RefFieldAccessorVisitor(usedFields: Array[Int]) extends RexVisitorImpl[Uni
             // access is top-level access => return top-level access
             case _ :: _ if nestedAccess.equals("*") => List("*")
             // previous access is not prefix of this access => add access
-            case head :: _ if !nestedAccess.startsWith(head) =>
+            // it may cause bug without "." as tail if we have references a.b and a.bb
+            case head :: _ if !nestedAccess.startsWith(head + ".") =>

Review comment:
       what if a field name contains `.` ?

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriter.scala
##########
@@ -60,3 +79,88 @@ class InputRewriter(fieldMap: Map[Int, Int]) extends RexShuttle {
     fieldMap.getOrElse(ref.getIndex,
       throw new IllegalArgumentException("input field contains invalid index"))
 }
+
+/**
+ * A RexShuttle to rewrite field accesses of RexNode with nested projection.
+ * For `RexInputRef`, it works like `InputReWriter` and use the old input
+ * ref index to find the new input fields ref.
+ * For `RexFieldAccess`, it will traverse to the top level of the access and
+ * find the mapping in field fieldMap first. There are 3 situations we need to consider:
+ *  1. mapping has the top level access, we should make field access to the reference;
+ *  2. mapping has the field, we should make an access;
+ *  3. mapping has no information of the current name, we should keep the full name
+ *  of the fields and index of mapping for later lookup.
+ * When the process is back from the recursion, we still have 2 situations need to
+ * consider:
+ *  1. we have found the reference of the upper level, we just make an access above the
+ *  reference we find before;
+ *  2. we haven't found the reference of the upper level, we concatenate the prefix with
+ *  the current field name and look up the new prefix in the mapping. If it's in the mapping,
+ *  we create a reference. Otherwise, we should go to the next level with the new prefix.
+ */
+class NestedInputRewriter(
+  fieldMap: JMap[Integer, JMap[String, Integer]],
+  rowTypes: JList[RelDataType],
+  builder: RexBuilder) extends RexShuttle {
+
+  override def visitFieldAccess(input: RexFieldAccess): RexNode = {
+    def traverse(fieldAccess: RexFieldAccess): (Int, String, Option[RexNode]) = {
+      fieldAccess.getReferenceExpr match {
+        case ref: RexInputRef =>
+          val mapping =
+            fieldMap.getOrElse(ref.getIndex,
+              throw new IllegalArgumentException("input field contains unknown index"))
+          if (mapping.contains("*")) {
+            (ref.getIndex,
+              "",
+              Option.apply(builder.makeFieldAccess(
+                new RexInputRef(mapping("*"), rowTypes(mapping("*"))),
+                fieldAccess.getField.getName,
+                false))
+            )
+          } else if(mapping.contains(fieldAccess.getField.getName)) {
+            (ref.getIndex,
+              "",
+              Option.apply(new RexInputRef(mapping(fieldAccess.getField.getName),

Review comment:
       ditto

##########
File path: flink-table/flink-table-planner-blink/src/test/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRuleTest.java
##########
@@ -107,4 +96,13 @@ private void testNestedProject(boolean nestedProjectionSupported) {
 		util().verifyPlan(sqlQuery);
 	}
 
+	@Test
+	public void testComplicatedNestedProject() {
+		String sqlQuery = "SELECT id," +
+				"    deepNested.nested1.name AS nestedName,\n" +
+				"    deepNested.nested2 AS nested2,\n" +
+				"    deepNested.nested2.num AS nestedNum\n" +
+				"FROM NestedTable";
+		util().verifyPlan(sqlQuery);

Review comment:
       nit: add a test about complex expressions, such as  deepNested.nested1.name + nested.value

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriter.scala
##########
@@ -60,3 +79,88 @@ class InputRewriter(fieldMap: Map[Int, Int]) extends RexShuttle {
     fieldMap.getOrElse(ref.getIndex,
       throw new IllegalArgumentException("input field contains invalid index"))
 }
+
+/**
+ * A RexShuttle to rewrite field accesses of RexNode with nested projection.
+ * For `RexInputRef`, it works like `InputReWriter` and use the old input
+ * ref index to find the new input fields ref.
+ * For `RexFieldAccess`, it will traverse to the top level of the access and
+ * find the mapping in field fieldMap first. There are 3 situations we need to consider:
+ *  1. mapping has the top level access, we should make field access to the reference;
+ *  2. mapping has the field, we should make an access;
+ *  3. mapping has no information of the current name, we should keep the full name
+ *  of the fields and index of mapping for later lookup.
+ * When the process is back from the recursion, we still have 2 situations need to
+ * consider:
+ *  1. we have found the reference of the upper level, we just make an access above the
+ *  reference we find before;
+ *  2. we haven't found the reference of the upper level, we concatenate the prefix with
+ *  the current field name and look up the new prefix in the mapping. If it's in the mapping,
+ *  we create a reference. Otherwise, we should go to the next level with the new prefix.
+ */
+class NestedInputRewriter(
+  fieldMap: JMap[Integer, JMap[String, Integer]],
+  rowTypes: JList[RelDataType],
+  builder: RexBuilder) extends RexShuttle {
+
+  override def visitFieldAccess(input: RexFieldAccess): RexNode = {
+    def traverse(fieldAccess: RexFieldAccess): (Int, String, Option[RexNode]) = {
+      fieldAccess.getReferenceExpr match {
+        case ref: RexInputRef =>
+          val mapping =
+            fieldMap.getOrElse(ref.getIndex,
+              throw new IllegalArgumentException("input field contains unknown index"))
+          if (mapping.contains("*")) {
+            (ref.getIndex,
+              "",
+              Option.apply(builder.makeFieldAccess(

Review comment:
       nit: Option.apply() -> Some()

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeRewriter.scala
##########
@@ -60,3 +79,88 @@ class InputRewriter(fieldMap: Map[Int, Int]) extends RexShuttle {
     fieldMap.getOrElse(ref.getIndex,
       throw new IllegalArgumentException("input field contains invalid index"))
 }
+
+/**
+ * A RexShuttle to rewrite field accesses of RexNode with nested projection.
+ * For `RexInputRef`, it works like `InputReWriter` and use the old input
+ * ref index to find the new input fields ref.
+ * For `RexFieldAccess`, it will traverse to the top level of the access and
+ * find the mapping in field fieldMap first. There are 3 situations we need to consider:
+ *  1. mapping has the top level access, we should make field access to the reference;
+ *  2. mapping has the field, we should make an access;
+ *  3. mapping has no information of the current name, we should keep the full name
+ *  of the fields and index of mapping for later lookup.
+ * When the process is back from the recursion, we still have 2 situations need to
+ * consider:
+ *  1. we have found the reference of the upper level, we just make an access above the
+ *  reference we find before;
+ *  2. we haven't found the reference of the upper level, we concatenate the prefix with
+ *  the current field name and look up the new prefix in the mapping. If it's in the mapping,
+ *  we create a reference. Otherwise, we should go to the next level with the new prefix.
+ */
+class NestedInputRewriter(
+  fieldMap: JMap[Integer, JMap[String, Integer]],
+  rowTypes: JList[RelDataType],
+  builder: RexBuilder) extends RexShuttle {
+
+  override def visitFieldAccess(input: RexFieldAccess): RexNode = {
+    def traverse(fieldAccess: RexFieldAccess): (Int, String, Option[RexNode]) = {
+      fieldAccess.getReferenceExpr match {
+        case ref: RexInputRef =>
+          val mapping =
+            fieldMap.getOrElse(ref.getIndex,
+              throw new IllegalArgumentException("input field contains unknown index"))
+          if (mapping.contains("*")) {
+            (ref.getIndex,
+              "",
+              Option.apply(builder.makeFieldAccess(
+                new RexInputRef(mapping("*"), rowTypes(mapping("*"))),
+                fieldAccess.getField.getName,
+                false))
+            )
+          } else if(mapping.contains(fieldAccess.getField.getName)) {
+            (ref.getIndex,
+              "",
+              Option.apply(new RexInputRef(mapping(fieldAccess.getField.getName),
+                rowTypes(mapping(fieldAccess.getField.getName)))))
+          } else {
+            (ref.getIndex, fieldAccess.getField.getName, Option.empty)
+          }
+        case acc: RexFieldAccess =>
+          val (i, prefix, node) = traverse(acc)
+          if (node.isDefined) {
+            (i,
+              "",
+              Option.apply(builder.makeFieldAccess(node.get, fieldAccess.getField.getName, false)))
+          } else {
+            val newPrefix = s"$prefix.${fieldAccess.getField.getName}"
+            // we have checked before
+            val mapping = fieldMap(i)
+            if (mapping.contains(newPrefix)) {
+              (i,
+                "",
+                Option.apply(new RexInputRef(mapping(newPrefix), rowTypes(mapping(newPrefix)))))
+            } else {
+              (i, newPrefix, Option.empty)
+            }
+          }
+      }
+    }

Review comment:
       it's better we can move this method out of its parent method, which could improve code readability 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 1697a20356799dd5e2f8863110de1f4626664f12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519) 
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709






----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 1697a20356799dd5e2f8863110de1f4626664f12 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519) 
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 9898b59d09c7483e3e0c034372a4875c46844841 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * da9bc2d9410e070611621051bcce5461443335a5 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe merged pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe merged pull request #13631:
URL: https://github.com/apache/flink/pull/13631


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708319526


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit 803f22afa05eac973c3c9e7b69722b4840c8173f (Wed Oct 14 10:41:56 UTC 2020)
   
   **Warnings:**
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] godfreyhe commented on a change in pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
godfreyhe commented on a change in pull request #13631:
URL: https://github.com/apache/flink/pull/13631#discussion_r516453143



##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the

Review comment:
       If `indexInNewSchema` is only used for leaf node ,  I suggest renamaing `indexInNewSchema` to `indexOfLeafInNewSchema`

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(
+    val columns: util.LinkedHashMap[String, RexNodeNestedField]) {
+
+}
+
+object RexNodeNestedFields {
+  /**
+   * It will uses the RexNodes to build a tree of the used fields.
+   * It uses a visitor to visit the operands of the expression. For
+   * input ref, it sits on the top level of the schema and it is the
+   * direct child of the root. For field access, it first decompose
+   * the field into a list and then create the node for every node in
+   * the list.
+   *
+   * In some situation, it will delete node. For example, the input
+   * expressions are "$0.child" and "$0". It will first create the
+   * intermediate node "$0" and leaf node "child". When coming to the
+   * expression "$0", it indicate the query will use the whole fields "$0"
+   * rather than the child "child" only. In this situation, it will mark
+   * the node "$0" as a leaf node and delete its children.
+   * */
+  def build(exprs: JList[RexNode], rowType: RelDataType):
+      RexNodeNestedFields = {
+    // the order field in the root node is to memorize
+    // the number of leaf
+    val root = new RexNodeNestedField(
+      "root",
+      -1,
+      rowType,
+      new util.LinkedHashMap[String, RexNodeNestedField](),
+      false,
+      -1)
+    val visitor = new NestedFieldExtractor(root, rowType)
+    for(expr <- exprs) {
+      expr.accept(visitor)
+    }
+    new RexNodeNestedFields(root.children)
+  }
+
+  /**
+   * After the projection, the used fields location has been changed.
+   * If the node in the tree has been labeled with the order, it will
+   * rewrite the location in the old schema with the new location.
+   *
+   * It uses a visitor to visit operands of the RexNode. If the type of
+   * operand is InputRef, it still in the top level of the schema and get
+   * the location of the fields using map. If the type of the operand is
+   * FieldAccess, it will first traverse to the top level of the field and
+   * iterate every level of the field with the name in the RexNode. For more
+   * details, please refer to NestedFieldReWriter.
+   */
+  def rewrite(
+      exprs: JList[RexNode],
+      root: RexNodeNestedFields,
+      builder: RexBuilder): JList[RexNode] = {
+    val writer = new NestedFieldReWriter(root, builder)
+    exprs.map(_.accept(writer)).toList.asJava
+  }
+
+  /**
+   * It will label the order of the leaf node with the insert order rather
+   * than the natural order of the name and output the path to the every
+   * leaf node. The paths are useful for interface SupportsProjectionPushDown
+   * and test(debug).
+   */
+  def labelAndConvert(root: RexNodeNestedFields): Array[Array[Int]] = {
+    val allPaths = new util.LinkedList[Array[Int]]()
+    val path = new util.LinkedList[Int]()
+    root.columns.foldLeft(0) {
+      case (newOrder, (_, column)) =>
+        traverse(column, newOrder, path, allPaths)
+    }
+    allPaths.toArray(new Array[Array[Int]](0))
+  }
+
+  private def traverse(
+      parent: RexNodeNestedField,
+      order: Int,

Review comment:
       index ?

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(
+    val columns: util.LinkedHashMap[String, RexNodeNestedField]) {
+
+}
+
+object RexNodeNestedFields {
+  /**
+   * It will uses the RexNodes to build a tree of the used fields.
+   * It uses a visitor to visit the operands of the expression. For
+   * input ref, it sits on the top level of the schema and it is the
+   * direct child of the root. For field access, it first decompose
+   * the field into a list and then create the node for every node in
+   * the list.
+   *
+   * In some situation, it will delete node. For example, the input
+   * expressions are "$0.child" and "$0". It will first create the
+   * intermediate node "$0" and leaf node "child". When coming to the
+   * expression "$0", it indicate the query will use the whole fields "$0"
+   * rather than the child "child" only. In this situation, it will mark
+   * the node "$0" as a leaf node and delete its children.
+   * */
+  def build(exprs: JList[RexNode], rowType: RelDataType):
+      RexNodeNestedFields = {
+    // the order field in the root node is to memorize
+    // the number of leaf
+    val root = new RexNodeNestedField(
+      "root",
+      -1,
+      rowType,
+      new util.LinkedHashMap[String, RexNodeNestedField](),
+      false,
+      -1)
+    val visitor = new NestedFieldExtractor(root, rowType)
+    for(expr <- exprs) {
+      expr.accept(visitor)
+    }
+    new RexNodeNestedFields(root.children)
+  }
+
+  /**
+   * After the projection, the used fields location has been changed.
+   * If the node in the tree has been labeled with the order, it will
+   * rewrite the location in the old schema with the new location.
+   *
+   * It uses a visitor to visit operands of the RexNode. If the type of
+   * operand is InputRef, it still in the top level of the schema and get
+   * the location of the fields using map. If the type of the operand is
+   * FieldAccess, it will first traverse to the top level of the field and
+   * iterate every level of the field with the name in the RexNode. For more
+   * details, please refer to NestedFieldReWriter.
+   */
+  def rewrite(
+      exprs: JList[RexNode],
+      root: RexNodeNestedFields,
+      builder: RexBuilder): JList[RexNode] = {
+    val writer = new NestedFieldReWriter(root, builder)
+    exprs.map(_.accept(writer)).toList.asJava
+  }
+
+  /**
+   * It will label the order of the leaf node with the insert order rather
+   * than the natural order of the name and output the path to the every
+   * leaf node. The paths are useful for interface SupportsProjectionPushDown
+   * and test(debug).
+   */
+  def labelAndConvert(root: RexNodeNestedFields): Array[Array[Int]] = {
+    val allPaths = new util.LinkedList[Array[Int]]()
+    val path = new util.LinkedList[Int]()
+    root.columns.foldLeft(0) {
+      case (newOrder, (_, column)) =>
+        traverse(column, newOrder, path, allPaths)
+    }
+    allPaths.toArray(new Array[Array[Int]](0))
+  }
+
+  private def traverse(
+      parent: RexNodeNestedField,
+      order: Int,
+      path: JList[Int],
+      allPaths: JList[Array[Int]]): Int ={
+    val tail = path.size()
+    // push self
+    path.add(parent.indexInOriginSchema)
+    val newOrder = if (parent.isLeaf) {
+      // leaf node
+      parent.indexInNewSchema = order
+      // ignore root node
+      allPaths.add(path.asScala.toArray)
+      order + 1
+    } else {
+      // iterate children
+      parent.children.values().foldLeft(order) {
+        case (newOrder, child) =>
+          traverse(child, newOrder, path, allPaths)
+      }
+    }
+    // pop self
+    path.remove(tail)
+    newOrder
+  }
+}
+
+/**
+ * A RexShuttle to rewrite field accesses of RexNode with nested projection.
+ * For `RexInputRef`, it uses the old input ref name to find the new input fields ref
+ * and use the order to generate the new input ref.
+ * For `RexFieldAccess`, it will traverse to the top level of the field access and
+ * then to generate new RexNode. There are 3 situations we need to consider:
+ *  1. if top level field is marked to use all sub-fields , make field access of the reference
+ *  and warp the ref as RexFieldAccess with the sub field name;
+ *  2. if top level field isn't marked to use all sub-fields and its direct field
+ *  is marked as useall, make field reference of the direct subfield;
+ *  3. if neither situation above happens, return from the recursion with the updated parent.
+ * When the process is back from the recursion, it still has 2 situations need to
+ * consider:
+ *  1. if the process has found the reference of the upper level, just make an access on the
+ *  reference founded before;
+ *  2. if the process hasn't found the first reference, the process continues to search under
+ *  the current parent.
+ */
+private class NestedFieldReWriter(
+    usedFields: RexNodeNestedFields,
+    builder: RexBuilder) extends RexShuttle {
+  override def visitInputRef(inputRef: RexInputRef): RexNode = {
+    if (!usedFields.columns.containsKey(inputRef.getName)) {
+      throw new TableException(
+        "Illegal input field access" + inputRef.getName)
+    } else {
+      val field = usedFields.columns.get(inputRef.getName)
+      new RexInputRef(field.indexInNewSchema, field.fieldType)
+    }
+  }
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): RexNode = {
+    val (node, _) = traverse(fieldAccess)
+    if (node.isDefined) {
+      node.get
+    } else {
+      throw new TableException(
+        "Unknown field " + fieldAccess + " when rewrite projection ")
+    }
+  }
+
+  private def traverse(
+      fieldAccess: RexFieldAccess): (Option[RexNode], RexNodeNestedField) = {
+    fieldAccess.getReferenceExpr match {
+      case ref: RexInputRef =>
+        val parent = usedFields.columns.get(ref.getName)
+        if (parent.isLeaf) {
+          (
+            Some(builder.makeFieldAccess(
+              new RexInputRef(parent.indexInNewSchema, parent.fieldType),
+              fieldAccess.getField.getName,
+              true)), parent)
+        } else {
+          val child = parent.children.get(fieldAccess.getField.getName)
+          if (child.isLeaf) {
+            (Some(new RexInputRef(child.indexInNewSchema, child.fieldType)), child)
+          } else {
+            (Option.empty, child)
+          }
+        }
+      case acc: RexFieldAccess =>
+        val (field, parent) = traverse(acc)
+        if (field.isDefined) {
+          (
+            Some(
+              builder.makeFieldAccess(
+                field.get,
+                fieldAccess.getField.getName,
+                true)),
+            parent)
+        } else {
+          val child = parent.children.get(fieldAccess.getField.getName)
+          if (child.isLeaf) {
+            (Some(new RexInputRef(child.indexInNewSchema, child.fieldType)), child)
+          } else {
+            (Option.empty, child)
+          }
+        }
+    }
+  }
+}
+
+/**
+ * An RexVisitor to extract all referenced input fields
+ */
+private class NestedFieldExtractor(val root: RexNodeNestedField, val rowType: RelDataType)

Review comment:
       use `RexNodeNestedFields`, and remove `val`s

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(
+    val columns: util.LinkedHashMap[String, RexNodeNestedField]) {
+
+}
+
+object RexNodeNestedFields {
+  /**
+   * It will uses the RexNodes to build a tree of the used fields.
+   * It uses a visitor to visit the operands of the expression. For
+   * input ref, it sits on the top level of the schema and it is the
+   * direct child of the root. For field access, it first decompose
+   * the field into a list and then create the node for every node in
+   * the list.
+   *
+   * In some situation, it will delete node. For example, the input
+   * expressions are "$0.child" and "$0". It will first create the
+   * intermediate node "$0" and leaf node "child". When coming to the
+   * expression "$0", it indicate the query will use the whole fields "$0"
+   * rather than the child "child" only. In this situation, it will mark
+   * the node "$0" as a leaf node and delete its children.
+   * */
+  def build(exprs: JList[RexNode], rowType: RelDataType):
+      RexNodeNestedFields = {

Review comment:
       nit: merge them into one line

##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedFieldTest.scala
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.planner.calcite.FlinkRexBuilder
+
+import java.util
+import java.util.{List => JList}
+
+import org.junit.Assert.{assertThat, assertTrue}
+import org.junit.Test
+
+import org.hamcrest.{BaseMatcher, Description}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ *  Test for RexNodeNestedField.
+ */
+class RexNodeNestedFieldTest extends RexNodeTestBase{
+  private class UnorderedArrayMatcher(expected: util.HashSet[JList[Int]])

Review comment:
       it does not compare the length of expected set and actual array

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(

Review comment:
       rename to `NestedSchema` ? and rename the file to `NestedSchema` too

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(
+    val columns: util.LinkedHashMap[String, RexNodeNestedField]) {
+
+}
+
+object RexNodeNestedFields {
+  /**
+   * It will uses the RexNodes to build a tree of the used fields.
+   * It uses a visitor to visit the operands of the expression. For
+   * input ref, it sits on the top level of the schema and it is the
+   * direct child of the root. For field access, it first decompose
+   * the field into a list and then create the node for every node in
+   * the list.
+   *
+   * In some situation, it will delete node. For example, the input
+   * expressions are "$0.child" and "$0". It will first create the
+   * intermediate node "$0" and leaf node "child". When coming to the
+   * expression "$0", it indicate the query will use the whole fields "$0"
+   * rather than the child "child" only. In this situation, it will mark
+   * the node "$0" as a leaf node and delete its children.
+   * */
+  def build(exprs: JList[RexNode], rowType: RelDataType):
+      RexNodeNestedFields = {
+    // the order field in the root node is to memorize
+    // the number of leaf
+    val root = new RexNodeNestedField(
+      "root",
+      -1,
+      rowType,
+      new util.LinkedHashMap[String, RexNodeNestedField](),
+      false,
+      -1)
+    val visitor = new NestedFieldExtractor(root, rowType)
+    for(expr <- exprs) {
+      expr.accept(visitor)
+    }
+    new RexNodeNestedFields(root.children)
+  }
+
+  /**
+   * After the projection, the used fields location has been changed.
+   * If the node in the tree has been labeled with the order, it will
+   * rewrite the location in the old schema with the new location.
+   *
+   * It uses a visitor to visit operands of the RexNode. If the type of
+   * operand is InputRef, it still in the top level of the schema and get
+   * the location of the fields using map. If the type of the operand is
+   * FieldAccess, it will first traverse to the top level of the field and
+   * iterate every level of the field with the name in the RexNode. For more
+   * details, please refer to NestedFieldReWriter.
+   */
+  def rewrite(
+      exprs: JList[RexNode],
+      root: RexNodeNestedFields,
+      builder: RexBuilder): JList[RexNode] = {
+    val writer = new NestedFieldReWriter(root, builder)
+    exprs.map(_.accept(writer)).toList.asJava
+  }
+
+  /**
+   * It will label the order of the leaf node with the insert order rather
+   * than the natural order of the name and output the path to the every
+   * leaf node. The paths are useful for interface SupportsProjectionPushDown
+   * and test(debug).
+   */
+  def labelAndConvert(root: RexNodeNestedFields): Array[Array[Int]] = {
+    val allPaths = new util.LinkedList[Array[Int]]()
+    val path = new util.LinkedList[Int]()
+    root.columns.foldLeft(0) {
+      case (newOrder, (_, column)) =>
+        traverse(column, newOrder, path, allPaths)
+    }
+    allPaths.toArray(new Array[Array[Int]](0))
+  }
+
+  private def traverse(
+      parent: RexNodeNestedField,
+      order: Int,
+      path: JList[Int],
+      allPaths: JList[Array[Int]]): Int ={
+    val tail = path.size()
+    // push self
+    path.add(parent.indexInOriginSchema)
+    val newOrder = if (parent.isLeaf) {
+      // leaf node
+      parent.indexInNewSchema = order
+      // ignore root node
+      allPaths.add(path.asScala.toArray)
+      order + 1
+    } else {
+      // iterate children
+      parent.children.values().foldLeft(order) {
+        case (newOrder, child) =>
+          traverse(child, newOrder, path, allPaths)
+      }
+    }
+    // pop self
+    path.remove(tail)
+    newOrder
+  }
+}
+
+/**
+ * A RexShuttle to rewrite field accesses of RexNode with nested projection.
+ * For `RexInputRef`, it uses the old input ref name to find the new input fields ref
+ * and use the order to generate the new input ref.
+ * For `RexFieldAccess`, it will traverse to the top level of the field access and
+ * then to generate new RexNode. There are 3 situations we need to consider:
+ *  1. if top level field is marked to use all sub-fields , make field access of the reference
+ *  and warp the ref as RexFieldAccess with the sub field name;
+ *  2. if top level field isn't marked to use all sub-fields and its direct field
+ *  is marked as useall, make field reference of the direct subfield;
+ *  3. if neither situation above happens, return from the recursion with the updated parent.
+ * When the process is back from the recursion, it still has 2 situations need to
+ * consider:
+ *  1. if the process has found the reference of the upper level, just make an access on the
+ *  reference founded before;
+ *  2. if the process hasn't found the first reference, the process continues to search under
+ *  the current parent.
+ */
+private class NestedFieldReWriter(
+    usedFields: RexNodeNestedFields,
+    builder: RexBuilder) extends RexShuttle {
+  override def visitInputRef(inputRef: RexInputRef): RexNode = {
+    if (!usedFields.columns.containsKey(inputRef.getName)) {
+      throw new TableException(
+        "Illegal input field access" + inputRef.getName)
+    } else {
+      val field = usedFields.columns.get(inputRef.getName)
+      new RexInputRef(field.indexInNewSchema, field.fieldType)
+    }
+  }
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): RexNode = {
+    val (node, _) = traverse(fieldAccess)
+    if (node.isDefined) {
+      node.get
+    } else {
+      throw new TableException(
+        "Unknown field " + fieldAccess + " when rewrite projection ")
+    }
+  }
+
+  private def traverse(
+      fieldAccess: RexFieldAccess): (Option[RexNode], RexNodeNestedField) = {
+    fieldAccess.getReferenceExpr match {
+      case ref: RexInputRef =>
+        val parent = usedFields.columns.get(ref.getName)
+        if (parent.isLeaf) {
+          (
+            Some(builder.makeFieldAccess(
+              new RexInputRef(parent.indexInNewSchema, parent.fieldType),
+              fieldAccess.getField.getName,
+              true)), parent)
+        } else {
+          val child = parent.children.get(fieldAccess.getField.getName)
+          if (child.isLeaf) {
+            (Some(new RexInputRef(child.indexInNewSchema, child.fieldType)), child)
+          } else {
+            (Option.empty, child)
+          }
+        }
+      case acc: RexFieldAccess =>
+        val (field, parent) = traverse(acc)
+        if (field.isDefined) {
+          (
+            Some(
+              builder.makeFieldAccess(
+                field.get,
+                fieldAccess.getField.getName,
+                true)),
+            parent)
+        } else {
+          val child = parent.children.get(fieldAccess.getField.getName)
+          if (child.isLeaf) {
+            (Some(new RexInputRef(child.indexInNewSchema, child.fieldType)), child)
+          } else {
+            (Option.empty, child)
+          }
+        }
+    }
+  }
+}
+
+/**
+ * An RexVisitor to extract all referenced input fields
+ */
+private class NestedFieldExtractor(val root: RexNodeNestedField, val rowType: RelDataType)
+  extends RexVisitorImpl[Unit](true) {
+
+  override def visitFieldAccess(fieldAccess: RexFieldAccess): Unit = {
+    def internalVisit(fieldAccess: RexFieldAccess): (Int, List[String]) = {
+      fieldAccess.getReferenceExpr match {
+        case ref: RexInputRef =>
+          (ref.getIndex, List(ref.getName, fieldAccess.getField.getName))
+        case fac: RexFieldAccess =>
+          val (i, n) = internalVisit(fac)
+          (i, n :+ fieldAccess.getField.getName)
+      }
+    }
+
+    // extract the info
+    val (index, names) = internalVisit(fieldAccess)
+    root.addChild(
+      new RexNodeNestedField(
+        names.get(0),
+        index,
+        rowType.getFieldList.get(index).getType,
+        new util.LinkedHashMap[String, RexNodeNestedField](),
+        false,
+        -1))

Review comment:
       extract the similar code into a comment method

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(

Review comment:
       rename to `NestedColumn` ?

##########
File path: flink-table/flink-table-planner-blink/src/test/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedFieldTest.scala
##########
@@ -0,0 +1,234 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.planner.calcite.FlinkRexBuilder
+
+import java.util
+import java.util.{List => JList}
+
+import org.junit.Assert.{assertThat, assertTrue}
+import org.junit.Test
+
+import org.hamcrest.{BaseMatcher, Description}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ *  Test for RexNodeNestedField.
+ */
+class RexNodeNestedFieldTest extends RexNodeTestBase{
+  private class UnorderedArrayMatcher(expected: util.HashSet[JList[Int]])
+    extends BaseMatcher[Array[Array[Int]]] {
+    override def matches(item: Any): Boolean = {
+      for (path <- item.asInstanceOf[Array[Array[Int]]]) {
+        if (!expected.contains(path.toSeq.asJava)) {
+          return false
+        }
+      }
+      true
+    }
+
+    override def describeTo(description: Description): Unit = {
+      description.appendValueList("", ",", "", expected)
+    }
+  }
+
+  @Test
+  def testExtractRefInputFields(): Unit = {
+    val (exprs, rowType) = buildExprs()
+    val nestedFields = RexNodeNestedFields.build(exprs, rowType)
+    val actual = RexNodeNestedFields.labelAndConvert(nestedFields)
+    val expected = new util.HashSet[JList[Int]]
+    expected.add(util.Arrays.asList(2))
+    expected.add(util.Arrays.asList(3))
+    expected.add(util.Arrays.asList(1))
+
+    assertThat(actual, new UnorderedArrayMatcher(expected))
+  }
+
+  @Test
+  def testExtractRefNestedInputFields(): Unit = {
+    val (rexProgram, rowType) = buildExprsWithNesting()
+
+    val nestedFields = RexNodeNestedFields.build(rexProgram, rowType)
+    val actual = RexNodeNestedFields.labelAndConvert(nestedFields)
+    val expected = new util.HashSet[JList[Int]]
+    expected.add(Array(1, 1).toSeq.asJava)
+    expected.add(Array(0).toSeq.asJava)

Review comment:
       the expected is  
   1,1  
   0, 
   0.
   we can use list or array to store the expected result, and before comparing, we can sort the expected collection and actual collection.

##########
File path: flink-table/flink-table-planner-blink/src/main/scala/org/apache/flink/table/planner/plan/utils/RexNodeNestedField.scala
##########
@@ -0,0 +1,332 @@
+/*
+ * 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.planner.plan.utils
+
+import org.apache.flink.table.api.TableException
+
+import org.apache.calcite.rel.`type`.RelDataType
+import org.apache.calcite.rex._
+
+import java.util
+import java.util.{List => JList}
+
+import scala.collection.JavaConversions._
+import scala.collection.JavaConverters._
+
+/**
+ * RexNodeNestedField is a tree node to build the used fields tree.
+ *
+ * @param name                 The name of the fields in the origin schema
+ * @param indexInOriginSchema  The index of the field in the origin schema.
+ *                             It only works for the RowType.
+ * @param fieldType            The type of the field. It is useful when
+ *                             rewriting the projections.
+ * @param isLeaf               Mark the field is the leaf node in the tree.
+ * @param children             Store the children of the field. It's safe
+ *                             to use name as the index because name is
+ *                             unique in every level. It uses the
+ *                             LinkedHashMap to keep the insert order.
+ *                             In some cases, it can reduce the cost of the
+ *                             reorder of the fields in query.
+ * @param indexInNewSchema     It is used by the leaf node to memorize the
+ *                             index in the new schema.
+ */
+class RexNodeNestedField(
+    val name: String,
+    val indexInOriginSchema: Int,
+    val fieldType: RelDataType,
+    val children: util.LinkedHashMap[String, RexNodeNestedField],
+    var isLeaf: Boolean,
+    var indexInNewSchema: Int) {
+
+  def addChild(field: RexNodeNestedField): Unit = {
+    if (!children.contains(field.name)) {
+      isLeaf = false
+      children.put(field.name, field)
+    }
+  }
+}
+
+/**
+ * RexNodeNestedFields could be regard as a table schema that represents
+ * a table's structure with field names and data types. It uses a
+ * LinkedHashMap to store the pairs of name: String and column: RexNodeNestedField.
+ *
+ * @param columns  Fields in the origin schema are used by the query.
+ */
+class RexNodeNestedFields(
+    val columns: util.LinkedHashMap[String, RexNodeNestedField]) {
+
+}
+
+object RexNodeNestedFields {
+  /**
+   * It will uses the RexNodes to build a tree of the used fields.
+   * It uses a visitor to visit the operands of the expression. For
+   * input ref, it sits on the top level of the schema and it is the
+   * direct child of the root. For field access, it first decompose
+   * the field into a list and then create the node for every node in
+   * the list.
+   *
+   * In some situation, it will delete node. For example, the input
+   * expressions are "$0.child" and "$0". It will first create the
+   * intermediate node "$0" and leaf node "child". When coming to the
+   * expression "$0", it indicate the query will use the whole fields "$0"
+   * rather than the child "child" only. In this situation, it will mark
+   * the node "$0" as a leaf node and delete its children.
+   * */
+  def build(exprs: JList[RexNode], rowType: RelDataType):
+      RexNodeNestedFields = {
+    // the order field in the root node is to memorize
+    // the number of leaf
+    val root = new RexNodeNestedField(
+      "root",
+      -1,
+      rowType,
+      new util.LinkedHashMap[String, RexNodeNestedField](),
+      false,
+      -1)
+    val visitor = new NestedFieldExtractor(root, rowType)
+    for(expr <- exprs) {
+      expr.accept(visitor)
+    }
+    new RexNodeNestedFields(root.children)
+  }
+
+  /**
+   * After the projection, the used fields location has been changed.
+   * If the node in the tree has been labeled with the order, it will
+   * rewrite the location in the old schema with the new location.
+   *
+   * It uses a visitor to visit operands of the RexNode. If the type of
+   * operand is InputRef, it still in the top level of the schema and get
+   * the location of the fields using map. If the type of the operand is
+   * FieldAccess, it will first traverse to the top level of the field and
+   * iterate every level of the field with the name in the RexNode. For more
+   * details, please refer to NestedFieldReWriter.
+   */
+  def rewrite(
+      exprs: JList[RexNode],
+      root: RexNodeNestedFields,
+      builder: RexBuilder): JList[RexNode] = {
+    val writer = new NestedFieldReWriter(root, builder)
+    exprs.map(_.accept(writer)).toList.asJava
+  }
+
+  /**
+   * It will label the order of the leaf node with the insert order rather
+   * than the natural order of the name and output the path to the every
+   * leaf node. The paths are useful for interface SupportsProjectionPushDown
+   * and test(debug).
+   */
+  def labelAndConvert(root: RexNodeNestedFields): Array[Array[Int]] = {
+    val allPaths = new util.LinkedList[Array[Int]]()
+    val path = new util.LinkedList[Int]()
+    root.columns.foldLeft(0) {
+      case (newOrder, (_, column)) =>

Review comment:
       newIndex




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19693][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * cb66dc97754abc1468b03508a41a4f3a34ce9572 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883",
       "triggerID" : "4bc536663a8a89fee69b58494c76b8e6f5e98c25",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * 4bc536663a8a89fee69b58494c76b8e6f5e98c25 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8883) 
   * 5f9d0abfa8ebf2ca3f29d1eeb69f898a6b4de915 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8252",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     }, {
       "hash" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8255",
       "triggerID" : "2fdb44da037e75537996e45b164f2a7968c78c73",
       "triggerType" : "PUSH"
     }, {
       "hash" : "da9bc2d9410e070611621051bcce5461443335a5",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8270",
       "triggerID" : "da9bc2d9410e070611621051bcce5461443335a5",
       "triggerType" : "PUSH"
     }, {
       "hash" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8519",
       "triggerID" : "1697a20356799dd5e2f8863110de1f4626664f12",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656",
       "triggerType" : "PUSH"
     }, {
       "hash" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8652",
       "triggerID" : "9898b59d09c7483e3e0c034372a4875c46844841",
       "triggerType" : "PUSH"
     }, {
       "hash" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8707",
       "triggerID" : "e0da25cfa5ae3e3a5944327f56e392590796e6a0",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8719",
       "triggerID" : "cb980798a022a35131fccd5c536a5f5519f38cba",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8723",
       "triggerID" : "ea9b932a3cfed3d2f08a212d4a9cd69dbd11ac6f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8729",
       "triggerID" : "8c12b6feeb38af9ce5aabdff168e602cd850351c",
       "triggerType" : "PUSH"
     }, {
       "hash" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8732",
       "triggerID" : "aa094733a6678ec72712da928dd457e79f3f81a9",
       "triggerType" : "PUSH"
     }, {
       "hash" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8745",
       "triggerID" : "5ae2ac7b55d791a68e3126f215c2ac60896e7625",
       "triggerType" : "PUSH"
     }, {
       "hash" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8802",
       "triggerID" : "ea637a3e8a33df6264ded320755ec0a397342f0b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8841",
       "triggerID" : "0707f7489676ee1a7e709479e47c6a3da5dd0e8b",
       "triggerType" : "PUSH"
     }, {
       "hash" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "status" : "CANCELED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897",
       "triggerID" : "bb84157d91d0f564a199c6af00209db8f51a7dda",
       "triggerType" : "PUSH"
     }, {
       "hash" : "fd8961181c3838b57b78d28c22029d581031763e",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907",
       "triggerID" : "fd8961181c3838b57b78d28c22029d581031763e",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * a5194ee25cf0c08ecd1d1e484b3f2e335ffb7656 UNKNOWN
   * bb84157d91d0f564a199c6af00209db8f51a7dda Azure: [CANCELED](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8897) 
   * fd8961181c3838b57b78d28c22029d581031763e Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=8907) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 4b3b9898ceeebc31777e2aa96d04b445cf578d87 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720) 
   * d1b917bb33f837bf04e8351affe7e108911e79b4 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * d1b917bb33f837bf04e8351affe7e108911e79b4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733) 
   * a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7803",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     }, {
       "hash" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858",
       "triggerID" : "46f99482ad7bea8e467130fe5dd92b61b7d3d444",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a7fe7dec5d7c869ccbc644bba45df241dd05e953",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * 46f99482ad7bea8e467130fe5dd92b61b7d3d444 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7858) 
   * a7fe7dec5d7c869ccbc644bba45df241dd05e953 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #13631: [FLINK-19639][table sql/planner]Support SupportsNestedProjectionPushD…

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #13631:
URL: https://github.com/apache/flink/pull/13631#issuecomment-708335709


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7600",
       "triggerID" : "803f22afa05eac973c3c9e7b69722b4840c8173f",
       "triggerType" : "PUSH"
     }, {
       "hash" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7603",
       "triggerID" : "cb66dc97754abc1468b03508a41a4f3a34ce9572",
       "triggerType" : "PUSH"
     }, {
       "hash" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "11b0dfc5a9a5e6aa335f070abfc6781569deb96d",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7720",
       "triggerID" : "4b3b9898ceeebc31777e2aa96d04b445cf578d87",
       "triggerType" : "PUSH"
     }, {
       "hash" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733",
       "triggerID" : "d1b917bb33f837bf04e8351affe7e108911e79b4",
       "triggerType" : "PUSH"
     }, {
       "hash" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 11b0dfc5a9a5e6aa335f070abfc6781569deb96d UNKNOWN
   * d1b917bb33f837bf04e8351affe7e108911e79b4 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=7733) 
   * a21d7f8f4269b52e8ff24365e6e9dbc5f3a9d906 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run travis` re-run the last Travis build
    - `@flinkbot run azure` re-run the last Azure build
   </details>


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org