godfreyhe commented on code in PR #20118: URL: https://github.com/apache/flink/pull/20118#discussion_r915444008
########## flink-table/flink-table-planner/src/main/java/org/apache/flink/table/planner/plan/rules/logical/PushProjectIntoTableSourceScanRule.java: ########## @@ -337,11 +339,71 @@ private RowType performPushDown( private List<RexNode> rewriteProjections( RelOptRuleCall call, TableSourceTable source, NestedSchema projectedSchema) { final LogicalProject project = call.rel(0); + List<RexNode> newProjects = project.getProjects(); + if (supportsProjectionPushDown(source.tableSource())) { - return NestedProjectionUtil.rewrite( - project.getProjects(), projectedSchema, call.builder().getRexBuilder()); - } else { - return project.getProjects(); + // if support project push down, then all input ref will be rewritten includes metadata + // columns. + newProjects = + NestedProjectionUtil.rewrite( + newProjects, projectedSchema, call.builder().getRexBuilder()); + } else if (supportsMetadata(source.tableSource())) { + // supportsMetadataProjection only. + // Note: why not reuse the NestedProjectionUtil to rewrite metadata projection? because + // it only works for sources which support projection push down. + List<Column.MetadataColumn> metadataColumns = + DynamicSourceUtils.extractMetadataColumns( + source.contextResolvedTable().getResolvedSchema()); + if (metadataColumns.size() > 0) { + Set<String> metaCols = + metadataColumns.stream().map(m -> m.getName()).collect(Collectors.toSet()); + + MetadataOnlyProjectionRewriter rewriter = + new MetadataOnlyProjectionRewriter( + project.getInput().getRowType(), source.getRowType(), metaCols); + + newProjects = + newProjects.stream() + .map(p -> p.accept(rewriter)) + .collect(Collectors.toList()); + } + } + + return newProjects; + } + + private class MetadataOnlyProjectionRewriter extends RexShuttle { Review Comment: nit: This class can be marked as `static` -- 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. To unsubscribe, e-mail: issues-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org