kristoffSC commented on code in PR #21393: URL: https://github.com/apache/flink/pull/21393#discussion_r1071287539
########## flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java: ########## @@ -0,0 +1,331 @@ +/* + * 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.codesplit; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.codesplit.JavaParser.BlockStatementContext; +import org.apache.flink.table.codesplit.JavaParser.StatementContext; + +import org.antlr.v4.runtime.CharStreams; +import org.antlr.v4.runtime.CommonTokenStream; +import org.antlr.v4.runtime.ParserRuleContext; +import org.antlr.v4.runtime.TokenStreamRewriter; +import org.antlr.v4.runtime.atn.PredictionMode; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; + +/** + * Extract statements from IFs, ELSEs and WILEs blocks making them smaller. + * + * <p>BlockStatementSplitter does not recognize if statement operates on local of class member + * variable. Because of that, code must be preprocessed by {@link DeclarationRewriter} which + * converts all local variables extracted as to member variables. + * + * <p><i>Before</i> + * + * <pre><code> + * while (counter > 0) { + * int localA = a + 1000; + * System.out.println(localA); + * if (a > 0) { + * b = a * 2; + * c = b * 2; + * System.out.println(b); + * } else { + * b = a * 3; + * System.out.println(b); + * } + * counter--; + * } + * + * </code></pre> + * + * <p><i>After</i> + * + * <pre><code> + * while (counter > 0) { + * myFun_whileBody0_0(int a); + * if (a > 0) { + * myFun_whileBody0_0_ifBody0(int a); + * } else { + * myFun_whileBody0_0_ifBody1(int a); + * } + * counter--; + * } + * </code></pre> + * + * <p>Where bodies of extracted "methods" are: + * + * <pre><code> + * myFun_whileBody0_0(int a) -> + * int localA = a + 1000; + * System.out.println(localA); + * </code></pre> + * + * <pre><code> + * myFun_whileBody0_0_ifBody0(int a) -> + * b = a * 2; + * c = b * 2; + * System.out.println(b); + * </code></pre> + * + * <pre><code> + * myFun_whileBody0_0_ifBody1(int a) -> + * b = a * 3; + * System.out.println(b); + * </code></pre> + */ +@Internal +public class BlockStatementSplitter { + + private final String code; + + private final String parameters; + + private BlockStatementVisitor visitor; + + /** + * Initialize new BlockStatementSplitter. + * + * @param code a code block that should be rewritten. + * @param parameters parameters definition that should be used for extracted methods. + */ + public BlockStatementSplitter(String code, String parameters) { + this.code = code; + this.parameters = parameters; + } + + /** + * This method extracts statements from IFs, ELSE's and WHILE blocks from block code used during + * initialization of this object. Every entry of returned map can be seen as new method nam (map + * key) and method's body. The block names will be prefixed with provided context. + * + * @param context prefix for extracted blocks. + * @return a map of block name to block statements mappings. The key can be interpreted as name + * of extracted block/method and corresponding List represents individual statements (block' + * lines) for this block. + */ + public Map<String, List<String>> extractBlocks(String context) { + + this.visitor = new BlockStatementVisitor(code, context, parameters); + JavaParser javaParser = new JavaParser(visitor.tokenStream); + javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL); + visitor.visitStatement(javaParser.statement()); + return visitor.getAllBlocks(); + } + + /** + * Rewrite code block that was used for this object initialization. + * + * @return a map which key represent rewritten block name and value represents rewritten code + * block, including calls to extracted methods + */ + public Map<String, String> rewriteBlock() { + visitor.rewrite(); + Map<String, String> rewriteBlocks = new HashMap<>(); + rewriteBlocks.put(visitor.context, visitor.rewriter.getText()); + return rewriteBlocks; + } + + private static void mergeBlocks( + Map<String, List<String>> mergeA, Map<String, List<String>> mergeB) { + + for (Entry<String, List<String>> entry : mergeA.entrySet()) { + mergeB.merge( + entry.getKey(), + entry.getValue(), + (aBlocks, bBlocks) -> { + List<String> merge = new ArrayList<>(aBlocks.size() + bBlocks.size()); + merge.addAll(aBlocks); + merge.addAll(bBlocks); + return merge; + }); + } + } + + private static class BlockStatementVisitor extends JavaParserBaseVisitor<Void> { + + private final List<BlockStatementVisitor> children = new ArrayList<>(); + + private final List<ParserRuleContext> extractedSingleBlocks = new ArrayList<>(); + + private final String context; + + private final CommonTokenStream tokenStream; + + private final TokenStreamRewriter rewriter; + + private final String parameters; + + private int counter = 0; + + private int bscCounter = 0; + + private BlockStatementVisitor(String code, String context, String parameters) { + this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code))); + this.rewriter = new TokenStreamRewriter(tokenStream); + this.context = context; + this.parameters = parameters; + } + + private BlockStatementVisitor( + TokenStreamRewriter rewriter, String context, String parameters) { + this.tokenStream = null; + this.rewriter = rewriter; + this.context = context; + this.parameters = parameters; + } + + @Override + public Void visitStatement(StatementContext ctx) { + + if (ctx.getChildCount() == 0) { + return null; + } + + if (ctx.WHILE() != null) { + for (StatementContext statement : ctx.statement()) { + BlockStatementVisitor visitor = + new BlockStatementVisitor( + rewriter, context + "_whileBody" + counter++, parameters); + children.add(visitor); + visitor.visitStatement(statement); + } + } else if (ctx.IF() != null) { + for (StatementContext sc : ctx.statement()) { + BlockStatementVisitor visitor = + new BlockStatementVisitor( + rewriter, context + "_ifBody" + counter++, parameters); + children.add(visitor); + visitor.visitStatement(sc); + } + + } else if (ctx.ELSE() != null) { + System.out.println(CodeSplitUtil.getContextString(ctx)); + } else { + if (ctx.block() != null) { + for (BlockStatementContext bsc : ctx.block().blockStatement()) { + if (bsc.statement() != null + && (bsc.statement().IF() != null + || bsc.statement().ELSE() != null + || bsc.statement().WHILE() != null)) { + + String localContext = + String.format("%s_%d", this.context, bscCounter++); + BlockStatementVisitor visitor = + new BlockStatementVisitor(rewriter, localContext, parameters); + visitor.addBlocks(extractedSingleBlocks); Review Comment: I'm assuming you are wondering about case similar to the example below: ``` public void myFun1(int[] a, int[] b) throws RuntimeException { if (a[0] == 0) { a[11] = b[0]; a[12] = b[0]; if (a[2] == 0) { a[21] = 1; a[22] = 1; } else { a[23] = b[2]; a[24] = b[2]; } a[13] = b[0]; a[14] = b[0]; } else { a[0] = b[0]; a[1] = b[1]; a[2] = b[2]; } } ``` In this case, the rewritten code will look like this: ``` public void myFun1(int[] a, int[] b) throws RuntimeException { if (a[0] == 0) { myFun1_rewriteGroup1(a, b); myFun1_ifBody0(a, b); } else { myFun1_ifBody1(a, b); } } void myFun1_ifBody0_0_ifBody0(int[] a, int[] b) throws RuntimeException { a[21] = 1; a[22] = 1; } void myFun1_ifBody0_0_ifBody1(int[] a, int[] b) throws RuntimeException { a[23] = b[2]; a[24] = b[2]; } void myFun1_rewriteGroup1(int[] a, int[] b) throws RuntimeException { myFun1_ifBody0_0(a, b); if (a[2] == 0) { myFun1_ifBody0_0_ifBody0(a, b); } else { myFun1_ifBody0_0_ifBody1(a, b); } } void myFun1_ifBody0_0(int[] a, int[] b) throws RuntimeException { a[11] = b[0]; a[12] = b[0]; } void myFun1_ifBody1(int[] a, int[] b) throws RuntimeException { a[0] = b[0]; a[1] = b[1]; a[2] = b[2]; } void myFun1_ifBody0(int[] a, int[] b) throws RuntimeException { a[13] = b[0]; a[14] = b[0]; } ``` First time when we hit `for (BlockStatementContext bsc : ctx.block().blockStatement()) {` and its IF/ELSE/WHILE check will be for block: ``` if (a[2] == 0) { a[21] = 1; a[22] = 1; } else { a[23] = b[2]; a[24] = b[2]; } ``` For this block we will create a new Visitor and pass `extractedSingleBlocks` array that contains single line statements `a[11] = b[0];` and `a[12] = b[0];`. When we call `visitStatement` on this new visitor, the ` } else if (ctx.IF() != null) {` will be true. The we will create new Visitors, one for TRUE and FALSE branch of that statement. Both with have empty `extractedSingleBlocks`. So lets take a look at `myFun1_ifBody0` context. At first we will have `a[11] = b[0];` and `a[12] = b[0];` added to the `extractedSingleBlocks` for this context. Then we will go into ` `for (BlockStatementContext bsc : ctx.block().blockStatement()) {`` where we will create new visitor, with new context and we will pass `a[11] = b[0];` and `a[12] = b[0];` to there. Finally those will land under `myFun1_ifBody0_0` which happens to be this new created Visitor. Then going back to `myFun1_ifBody0`, now it has its `extractedSingleBlocks` empty, since we just clear it. The next and finnal call for this context will be adding `a[13] = b[0];` and `a[14] = b[0];` to `extractedSingleBlocks` which are last statement to process for this block. For more details please take a look at my response for -> https://github.com/apache/flink/pull/21393#discussion_r1071233842 -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
