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 2022/11/24 20:05:39 UTC

[GitHub] [flink] kristoffSC opened a new pull request, #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

kristoffSC opened a new pull request, #21393:
URL: https://github.com/apache/flink/pull/21393

   Signed-off-by: Krzysztof Chmielewski <kr...@gmail.com>
   
   <!--
   *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
   
   *(For example: This pull request makes task deployment go through the blob server, rather than through RPC. That way we avoid re-transferring them on each deployment (during recovery).)*
   
   
   ## Brief change log
   
   *(for example:)*
     - *The TaskInfo is stored in the blob store on job creation time as a persistent artifact*
     - *Deployments RPC transmits only the blob storage reference*
     - *TaskManagers retrieve the TaskInfo from the blob cache*
   
   
   ## Verifying this change
   
   Please make sure both new and modified tests in this PR follows the conventions defined in our code quality guide: https://flink.apache.org/contributing/code-style-and-quality-common.html#testing
   
   *(Please pick either of the following options)*
   
   This change is a trivial rework / code cleanup without any test coverage.
   
   *(or)*
   
   This change is already covered by existing tests, such as *(please describe tests)*.
   
   *(or)*
   
   This change added tests and can be verified as follows:
   
   *(example:)*
     - *Added integration tests for end-to-end deployment with large payloads (100MB)*
     - *Extended integration test for recovery after master (JobManager) failure*
     - *Added test that validates that TaskInfo is transferred only once across recoveries*
     - *Manually verified the change by running a 4 node cluster with 2 JobManagers and 4 TaskManagers, a stateful streaming program, and killing one JobManager and two TaskManagers during the execution, verifying that recovery happens correctly.*
   
   ## 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, 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1384601282

   Hi @tsreaper 
   I believe I have replayed to all your comments and suggestions. 
   I also provided some more explanation about the logic in few comments:
   
   https://github.com/apache/flink/pull/21393#discussion_r1071233842
   https://github.com/apache/flink/pull/21393#discussion_r1071287539
   
   Please let me know if now logic is more clear and would you like me to add this to java doc or something?


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1089692781


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.

Review Comment:
   `WILEs` -> `WHILEs`



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * 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_0_1(a, b, c);
+ *     if (a > 0) {
+ *         myFun_0_1_2(a, b, c);
+ *     } else {
+ *         myFun_0_1_3(a, b, c);
+ *     }
+ *     counter--;
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ * myFun_0_1(int a, int b, int c) ->
+ *     int localA = a + 1000;
+ *     System.out.println(localA);
+ * </code></pre>
+ *
+ * <pre><code>
+ * myFun_0_1_3(int a, int b, int c) ->
+ *     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;
+    }
+
+    /**
+     * Rewrite code block that was used for this object initialization.
+     *
+     * @param context prefix for extracted blocks.
+     * @return a map which key represent rewritten block name and value represents rewritten code
+     *     block, including calls to extracted methods
+     */
+    public String rewriteBlock(String context) {
+
+        this.visitor = new BlockStatementVisitor(code, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement(), context);
+        visitor.rewrite();
+        return visitor.rewriter.getText();
+    }
+
+    /**
+     * 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 name
+     * (map key) and method's body (map value). The block names will be prefixed with provided
+     * context.
+     *
+     * @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() {
+
+        Map<String, List<String>> allBlocks = new HashMap<>(visitor.blocks.size());
+
+        for (Entry<String, List<ParserRuleContext>> entry : visitor.blocks.entrySet()) {
+
+            List<String> blocks =
+                    entry.getValue().stream()
+                            .map(CodeSplitUtil::getContextString)
+                            .collect(Collectors.toList());
+            allBlocks.put(entry.getKey(), blocks);
+        }
+
+        return allBlocks;
+    }
+
+    private static class BlockStatementVisitor {
+
+        private final Map<String, List<ParserRuleContext>> blocks = new HashMap<>();
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementVisitor(String code, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.parameters = parameters;
+        }
+
+        public void visitStatement(StatementContext ctx, String context) {
+
+            if (ctx.getChildCount() == 0 || getNumOfReturnOrJumpStatements(ctx) != 0) {
+                return;
+            }
+
+            if (ctx.block() == null) {
+                for (StatementContext statementContext : ctx.statement()) {
+                    String localContext = String.format("%s_%d", context, counter++);
+                    visitStatement(statementContext, localContext);
+                }
+            } else {
+                List<ParserRuleContext> extractedSingleBlocks = new ArrayList<>();
+                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", context, counter++);
+
+                        // if there is only one statement in the block, and it is not IF/ELSE/WHILE
+                        // statement
+                        // it's pointless to extract it into a separate function.
+                        if (canGroupAsSingleStatement(extractedSingleBlocks)) {
+                            List<ParserRuleContext> previous =
+                                    blocks.put(localContext, extractedSingleBlocks);
+                            if (previous != null) {
+                                throw new RuntimeException(
+                                        String.format(
+                                                "Overriding extracted block %s - this should not happen.",
+                                                context));
+                            }

Review Comment:
   `Preconditions.checkState(previous != null, String.format("Overriding..."));`



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables, IF and
+ * WHILE statements and extract new method for each group making them smaller.
+ *
+ * <p>BlockStatementGrouper 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>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *     myFun_rewriteGroup4(a, b);
+ *     myFun_rewriteGroup5(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup4 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_1_rewriteGroup3(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup5 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_1_rewriteGroup3 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(maxMethodLength, parameters);
+        CommonTokenStream tokenStream =
+                new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+        JavaParser javaParser = new JavaParser(tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        TokenStreamRewriter rewriter = new TokenStreamRewriter(tokenStream);
+        visitor.visitStatement(javaParser.statement(), context, rewriter);
+
+        visitor.rewrite();
+        Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups = visitor.groups;
+
+        Map<String, List<String>> groupStrings = new HashMap<>(groups.size());
+        for (Entry<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> group :
+                groups.entrySet()) {
+            List<String> collectedStringGroups =
+                    group.getValue().getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+
+            groupStrings.put(group.getKey(), collectedStringGroups);
+        }
+
+        return new RewriteGroupedCode(rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor {
+
+        private final Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups =
+                new HashMap<>();
+
+        private final long maxMethodLength;
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(long maxMethodLength, String parameters) {
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        public void visitStatement(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+
+            if (ctx.getChildCount() == 0) {
+                return;
+            }
+
+            // For these statements here we want to process all "branches" separately, for example
+            // TRUE and FALSE branch of IF/ELSE block.
+            // each statement can be rewritten and extracted.
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        String localContext = String.format("%s_%d", context, counter++);
+                        groupBlock(statement, localContext, rewriter);
+                    }
+                }
+            } else {
+                // The block did not start from IF/ELSE/WHILE statement
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx, context, rewriter);
+                }
+            }
+        }
+
+        // Group continuous block of statements together. If Statement is an IF/ELSE/WHILE,
+        // its body can be further grouped by recursive call to visitStatement method.
+        private void groupBlock(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + this.counter++;
+
+                    CommonTokenStream tokenStream =
+                            new CommonTokenStream(
+                                    new JavaLexer(
+                                            CharStreams.fromString(
+                                                    CodeSplitUtil.getContextString(statement))));

Review Comment:
   You say "In my case we parse class once" in a previous comment, but here you repeatedly and recursively parse the blocks.



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,281 @@
+/*
+ * 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.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * 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_0_1(a, b, c);
+ *     if (a > 0) {
+ *         myFun_0_1_2(a, b, c);
+ *     } else {
+ *         myFun_0_1_3(a, b, c);
+ *     }
+ *     counter--;
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ * myFun_0_1(int a, int b, int c) ->
+ *     int localA = a + 1000;
+ *     System.out.println(localA);
+ * </code></pre>
+ *
+ * <pre><code>
+ * myFun_0_1_3(int a, int b, int c) ->
+ *     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;
+    }
+
+    /**
+     * Rewrite code block that was used for this object initialization.
+     *
+     * @param context prefix for extracted blocks.
+     * @return a map which key represent rewritten block name and value represents rewritten code
+     *     block, including calls to extracted methods
+     */
+    public String rewriteBlock(String context) {
+
+        this.visitor = new BlockStatementVisitor(code, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement(), context);
+        visitor.rewrite();
+        return visitor.rewriter.getText();
+    }
+
+    /**
+     * 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 name
+     * (map key) and method's body (map value). The block names will be prefixed with provided
+     * context.
+     *
+     * @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() {
+
+        Map<String, List<String>> allBlocks = new HashMap<>(visitor.blocks.size());
+
+        for (Entry<String, List<ParserRuleContext>> entry : visitor.blocks.entrySet()) {
+
+            List<String> blocks =
+                    entry.getValue().stream()
+                            .map(CodeSplitUtil::getContextString)
+                            .collect(Collectors.toList());
+            allBlocks.put(entry.getKey(), blocks);
+        }
+
+        return allBlocks;
+    }
+
+    private static class BlockStatementVisitor {
+
+        private final Map<String, List<ParserRuleContext>> blocks = new HashMap<>();
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementVisitor(String code, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.parameters = parameters;
+        }
+
+        public void visitStatement(StatementContext ctx, String context) {
+
+            if (ctx.getChildCount() == 0 || getNumOfReturnOrJumpStatements(ctx) != 0) {
+                return;
+            }
+
+            if (ctx.block() == null) {
+                for (StatementContext statementContext : ctx.statement()) {
+                    String localContext = String.format("%s_%d", context, counter++);
+                    visitStatement(statementContext, localContext);
+                }
+            } else {
+                List<ParserRuleContext> extractedSingleBlocks = new ArrayList<>();
+                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", context, counter++);
+
+                        // if there is only one statement in the block, and it is not IF/ELSE/WHILE
+                        // statement
+                        // it's pointless to extract it into a separate function.
+                        if (canGroupAsSingleStatement(extractedSingleBlocks)) {
+                            List<ParserRuleContext> previous =
+                                    blocks.put(localContext, extractedSingleBlocks);
+                            if (previous != null) {
+                                throw new RuntimeException(
+                                        String.format(
+                                                "Overriding extracted block %s - this should not happen.",
+                                                context));
+                            }
+                        }
+
+                        extractedSingleBlocks = new ArrayList<>();
+                        visitStatement(bsc.statement(), localContext);
+                    } else {
+                        extractedSingleBlocks.add(bsc);
+                    }
+                }
+                if (canGroupAsSingleStatement(extractedSingleBlocks)) {
+                    List<ParserRuleContext> previous = blocks.put(context, extractedSingleBlocks);
+                    if (previous != null) {
+                        throw new RuntimeException(
+                                String.format(
+                                        "Overriding extracted block %s - this should not happen.",
+                                        context));
+                    }
+                }

Review Comment:
   This block of code is the same as previous. Make them a 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1415919713

   Hi @tsreaper 
   I've implemented test you requested for. It's 'CodeSplitITCase ::testManyAggregationsWithGroupBy'
   
   CI/CD is green.
   
   Anything else you would like me to do? If not could you merge this PR?
   
   P.S.
   After merging this PR I would like to create a back port PRs to 1.15 and 1.16. Our client is on 1.14 which is not supported anymore and I'm not sure to which version our client will be upgrading to.
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1067994477


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;

Review Comment:
   This block will be replaced by a method call. Add the length of method call to `localGroupCodeLength`?



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);

Review Comment:
   You use recursions both in `BlockStatementSplitter#extractBlocks` and in `BlockStatementVisitor#visitStatement`. This doesn't look right to me. Could you explain your logic in detail?



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);
+
+                        mergeBlocks(rewrite, extractedBlocks);
+                    }
+                }
+            }
+        }
+
+        Map<String, List<String>> localBlocks = visitor.getAllBlocks();
+        mergeBlocks(localBlocks, extractedBlocks);
+
+        return extractedBlocks;
+    }
+
+    /**
+     * 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() {
+
+        for (BlockStatementSplitter child : children) {
+            child.visitor.rewrite();
+        }
+        visitor.rewrite();
+
+        Map<String, String> rewriteBlocks = new HashMap<>();
+        for (BlockStatementSplitter child : children) {
+            Map<String, String> childRewriteBlocks = child.rewriteBlock();
+            for (Entry<String, String> entry : childRewriteBlocks.entrySet()) {
+                rewriteBlocks.merge(
+                        entry.getKey(),
+                        entry.getValue(),
+                        (s, s2) -> {
+                            throw new RuntimeException(
+                                    String.format(
+                                            "Override rewritten block  for key %s. Blocks are %s -> %s",
+                                            entry.getKey(), s, s2));
+                        });
+            }
+        }
+
+        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<ContextTextPair> 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 {

Review Comment:
   What about `ctx.ELSE() != null`?



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1396174097

   Hi @tsreaper 
   Thank you for your valuable feedback, no need to apologize :) Please see my response below. 
   
   > In BlockStatementSplitter you group all single statements between IF/ELSE/WHILEs together and extract each group into a separate method.
   
   I'm not only extracting single line statements but also entire if/else calls. Other than that your understanding is correct.
   
   > (If my guesses are correct, you can implement this with just one for loop and a recursive call. No need for mergeBlocks or addGroups or something else. mergeBlocks and addGroups confuse me quick a lot.)
   
   I think I'm already doing this. There is a for loop for block statements. Every block has its own visitor. There is no recursion on calling visit method. 
   The only recursion is to traverse through all visitors and get blocks from them. This is straightforward parent -> child structure. 
   I'm not merging any blocks, I'm simply gathering all blocks from all visitors. Please take a look at my comment -> https://github.com/apache/flink/pull/21393#discussion_r1071174072
   
   Also please note that `addGroups` method was deleted. 
   
   
   > The problem of this idea (or rather, the current implementation) is that, it's doing almost the same thing with the old IfStatementRewriter, plus the currently remaining FunctionSplitter.
   
   I don't agree with that statement. Firstly my implementation handle `else if` blocks which original statement did not. Also in some cases, my implementation produced better results than original  `IfStatementRewriter`. In other cases - the result was the same as original `IfStatementRewriter`, So we have extra value here. Please look at more examples provided by me at the end of this comment.
   Also I'm optimizing and grouping IF/ELSE/WHILE branches bodies (block, statements) whereas `FunctionSplitter` assumes that statements is already optimized. That means that `FunctionSplitter` takesentire IF/Else without any extra modifications and adds it to the methods body or extracts it to the new method. So I'm not duplicating `FunctionSplitter`, those operate on different levels.
   
   > I guess what you need is just to copy a bit of code in IfStatementRewriter, change them into WHILE and that's all. Maybe changing the name to BlockStatementRewriter. This shall achieve our goals by making the least changes.
   
   I do not agree with this statement. `IfStatementRewriter` is specialized in IF/ELSE. It does not process "else if". 
   Originally I was thinking about adding bits of code to `IfStatementRewriter` but I realized that the problem is more complex. We need something that can process nested combination of WHILE's and IF's 
   I couldn't have things like 
   `shouldExtract(blockStatementContext.statement().statement(0))` and `shouldExtract(blockStatementContext.statement().statement(1))` 
   I wanted to have more generic code, that will apply the same logic to all blocks. In fact, one may argue that I actually did ` copy a bit of code in IfStatementRewriter change them into WHILE and that's all. ` You will find bits of `IfStatementRewriter` in "BlockStatementRewriter::rewrite` and "BlockStatementVisitor::visitMethodDeclaration` methods. Other than that, I proposed new solution.
   
   Also `IfStatementRewriter` approach for handling nested IF/ELSE is different that in my implementation. If I understand correctly, `IfStatementRewriter` rewrite entire class as many times as there were extracted methods.
   ```
           while (visitor.hasRewrite()) {
               visitor = new IfStatementVisitor(rewriterCode);
               rewriterCode = visitor.rewriteAndGetCode();
           }
   ```
   The first rewrite call will extract first method, `visitor.hasRewrite()` will return true, we will rewrite again, but this time we will rewrite the previously extracted method, and so on. Every time, the entire class has to be "parsed".
    
   In my case we parse class once, extracting new blocks and processing nested expressions by creating new visitors <- NO recursion here. Every visitor has its context, that will become a name prefix of extracted by this visitors. Every extracted block 
    
   Every new visitor has the same reference to "rewriter" object. This means that every visitor handles only small portion of entire method, and rewrites only its part.
    
    I do understand that this change touches core component, so the possible impact is big -> SQL jobs.
    I can tell that existing integration/end 2 end tests detected issues with previous implementations. I managed to solve those by analyzing original and rewritten code created for test queries. 
    
    To sum up:
    1. recursion -> I really don't think that this should be a problem. Recursion is called only to gather the results, parsing is not using recursion.
    2. addBlocks -> removed
    3. adding bits to `IfStatementRewriter` I dont think this is the option.
    
    I can try to implement your way of processing nested expressions, something similar to `while (visitor.hasRewrite())` from `IfStatementRewriter` if you think that can help pushing this PR.
    
   At the end please take a look at some examples:
    Example 1:
    Original code:
   ```
        public void myFun1(int[] a, int[] b) throws RuntimeException {
           if (a[0] == 0) {
               a[0] = 1;
               a[1] = 1;
           } else if (a[1] = 22) {
               a[1] = b[12];
               a[2] = b[22];
           } else if (a[3] == 0) {
               a[3] = b[3];
               a[33] = b[33];
           } else if (a[4] = 0) {
               a[4] = b[4];
               a[44] = b[44];
           } else {
               a[0] = b[0];
               a[1] = b[1];
               a[2] = b[2];
           }
       }
   ```
   
   BlockStatementRewriter result:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
   
           if (a[0] == 0) {
               myFun1_ifBody0(a, b);
           } else if (a[1] = 22) {
               myFun1_ifBody1_ifBody0(a, b);
           } else if (a[3] == 0) {
               myFun1_ifBody1_ifBody1_ifBody0(a, b);
           } else if (a[4] = 0) {
               myFun1_ifBody1_ifBody1_ifBody1_ifBody0(a, b);
           } else {
               myFun1_ifBody1_ifBody1_ifBody1_ifBody1(a, b);
           }
       }
   
       void myFun1_ifBody1_ifBody1_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[3] = b[3];
           a[33] = b[33];
       }
   
       void myFun1_ifBody1_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[1] = b[12];
           a[2] = b[22];
       }
   
       void myFun1_ifBody1_ifBody1_ifBody1_ifBody1(int[] a, int[] b) throws RuntimeException {
           a[0] = b[0];
           a[1] = b[1];
           a[2] = b[2];
       }
   
       void myFun1_ifBody1_ifBody1_ifBody1_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[4] = b[4];
           a[44] = b[44];
       }
   
       void myFun1_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[0] = 1;
           a[1] = 1;
       }
   ```
   
   IfStatementRewriter result:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
           if (a[0] == 0) {
               myFun1_trueFilter1(a, b);
   
           }
           else if (a[1] = 22) {
               a[1] = b[12];
               a[2] = b[22];
           } else if (a[3] == 0) {
               a[3] = b[3];
               a[33] = b[33];
           } else if (a[4] = 0) {
               a[4] = b[4];
               a[44] = b[44];
           } else {
               a[0] = b[0];
               a[1] = b[1];
               a[2] = b[2];
           }
       }
       void myFun1_trueFilter1(int[] a, int[] b) throws RuntimeException{
           a[0] = 1;
           a[1] = 1;
       }
   ```
   
   Example 2:
   Original code:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
           if (a[0] == 0) {
               System.out.println("0");
               System.out.println("0");
               if (a[1] == 0) {
                   System.out.println("1");
                   System.out.println("2");
                   if (a[2] == 0) {
                       a[2] = 1;
                       a[22] = 1;
                   } else {
                       a[2] = b[2];
                       a[22] = b[2];
                   }
               } else {
                   a[1] = b[1];
                   a[2] = b[2];
               }
           } else {
               System.out.println("3");
               System.out.println("3");
               if (a[1] == 1) {
                   a[0] = b[0];
                   a[1] = b[1];
                   a[2] = b[2];
               } else {
                   a[0] = 2 * b[0];
                   a[1] = 2 * b[1];
                   a[2] = 2 * b[2];
               }
           }
       }
   ```
   BlockStatementRewriter result:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
   
           if (a[0] == 0) {
               myFun1_rewriteGroup1(a, b);
           } else {
               myFun1_rewriteGroup3(a, b);
           }
       }
   
       void myFun1_ifBody0_0_ifBody0_0_ifBody1(int[] a, int[] b) throws RuntimeException {
           a[2] = b[2];
           a[22] = b[2];
       }
   
       void myFun1_ifBody0_0_ifBody0_0_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[2] = 1;
           a[22] = 1;
       }
   
       void myFun1_ifBody0_0_ifBody1(int[] a, int[] b) throws RuntimeException {
           a[1] = b[1];
           a[2] = b[2];
       }
   
       void myFun1_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
           myFun1_ifBody0_0(a, b);
           if (a[1] == 0) {
               myFun1_rewriteGroup0_rewriteGroup1(a, b);
           } else {
               myFun1_ifBody0_0_ifBody1(a, b);
           }
       }
   
       void myFun1_ifBody1_0(int[] a, int[] b) throws RuntimeException {
           System.out.println("3");
           System.out.println("3");
       }
   
       void myFun1_ifBody0_0(int[] a, int[] b) throws RuntimeException {
           System.out.println("0");
           System.out.println("0");
       }
   
       void myFun1_ifBody0_0_ifBody0_0(int[] a, int[] b) throws RuntimeException {
           System.out.println("1");
           System.out.println("2");
       }
   
       void myFun1_rewriteGroup3(int[] a, int[] b) throws RuntimeException {
           myFun1_ifBody1_0(a, b);
           if (a[1] == 1) {
               myFun1_ifBody1_0_ifBody0(a, b);
           } else {
               myFun1_ifBody1_0_ifBody1(a, b);
           }
       }
   
       void myFun1_ifBody1_0_ifBody0(int[] a, int[] b) throws RuntimeException {
           a[0] = b[0];
           a[1] = b[1];
           a[2] = b[2];
       }
   
       void myFun1_ifBody1_0_ifBody1(int[] a, int[] b) throws RuntimeException {
           a[0] = 2 * b[0];
           a[1] = 2 * b[1];
           a[2] = 2 * b[2];
       }
   
       void myFun1_rewriteGroup0_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
           myFun1_ifBody0_0_ifBody0_0(a, b);
           if (a[2] == 0) {
               myFun1_ifBody0_0_ifBody0_0_ifBody0(a, b);
           } else {
               myFun1_ifBody0_0_ifBody0_0_ifBody1(a, b);
           }
       }
   ```
   
   IfStatementRewriter result:
     ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
           if (a[0] == 0) {
               myFun1_trueFilter1(a, b);
   
           }
           else {
               myFun1_falseFilter2(a, b);
   
           }
   
       }
       void myFun1_trueFilter1(int[] a, int[] b) throws RuntimeException{
           System.out.println("0");
           System.out.println("0");
           if (a[1] == 0) {
               myFun1_trueFilter1_trueFilter3(a, b);
   
           }
           else {
               myFun1_trueFilter1_falseFilter4(a, b);
   
           }
   
       }
       void myFun1_trueFilter1_trueFilter3(int[] a, int[] b) throws RuntimeException{
           System.out.println("1");
           System.out.println("2");
           if (a[2] == 0) {
               myFun1_trueFilter1_trueFilter3_trueFilter7(a, b);
   
           }
           else {
               myFun1_trueFilter1_trueFilter3_falseFilter8(a, b);
   
           }
   
       }
       void myFun1_trueFilter1_trueFilter3_trueFilter7(int[] a, int[] b) throws RuntimeException{
           a[2] = 1;
           a[22] = 1;
       }
   
   
       void myFun1_trueFilter1_trueFilter3_falseFilter8(int[] a, int[] b) throws RuntimeException{
           a[2] = b[2];
           a[22] = b[2];
       }
   
   
   
   
       void myFun1_trueFilter1_falseFilter4(int[] a, int[] b) throws RuntimeException{
           a[1] = b[1];
           a[2] = b[2];
       }
   
   
   
   
       void myFun1_falseFilter2(int[] a, int[] b) throws RuntimeException{
           System.out.println("3");
           System.out.println("3");
           if (a[1] == 1) {
               myFun1_falseFilter2_trueFilter5(a, b);
   
           }
           else {
               myFun1_falseFilter2_falseFilter6(a, b);
   
           }
   
       }
       void myFun1_falseFilter2_trueFilter5(int[] a, int[] b) throws RuntimeException{
           a[0] = b[0];
           a[1] = b[1];
           a[2] = b[2];
       }
   
   
       void myFun1_falseFilter2_falseFilter6(int[] a, int[] b) throws RuntimeException{
           a[0] = 2 * b[0];
           a[1] = 2 * b[1];
           a[2] = 2 * b[2];
       }
   ```
   
   Cheers,
   And happy New Year ;)


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1085531144


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   @tsreaper 
   Sorry for delay on this one.
   
   Regarding:
   > I mean something like the following.
   
   I understand your suggestion, I was thinking about similar thing at the beginning of designing the solution.
   
   Let me try to rephrase what I think is main difference in yours and mine proposition.
   
   You are proposing to have single Visitor instance that will recursively parse every statement/block.  Every statement extracted from particular "level" will be added to `blocks` map, where key is "context" that identifies the "level/block" that is currently processed.
   
   In my proposition, I'm creating new Visitor for every new nested "block/level". Every visitor maintains its own list of extracted statements. At the end I'm traversing through all visitors and gather all extracted blocks -> mergeBlock method (not the best name I agree).
   
   
   Also why I had the initial ` if (ctx.WHILE() != null) {` and ` } else if (ctx.IF() != null) {` checks were actually was only to set context name to `IF` or `WHILE` i kinda thought it would be beneficial for debugging but I guess it adds the complexity here.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1408677439

   @tsreaper 
   Regarding `I'd like to see more examples. Could you please provide examples and explain` 
   
   If we focus only on IF/ELSE statements without `else if` blocks which are also supported in proposed solution I think that the differences boils down to one example below. 
   
   Having:
   ```
   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];
           }
   }
   ```
   
   The original `IfStatementRewritter will not extract: 
   ```
               a[11] = b[0];
               a[12] = b[0];
   ```
   and 
   ```
               a[13] = b[0];
               a[14] = b[0];
   ```
   
   To their's separate methods. They will be extracted together with entire `TRUE` branch code block. In my proposed solution, the `TRUE` branch will be extracted plus statements from above will be further extracted to their own methods like so:
   
   ```
       void myFun1_0_1(int[] a, int[] b) throws RuntimeException {
           a[11] = b[0];
           a[12] = b[0];
       }
   
       void myFun1_0(int[] a, int[] b) throws RuntimeException {
           a[13] = b[0];
           a[14] = b[0];
       }
   ```
   The test with similar code is implemented in: `BlockStatementRewriterTest::testIfMultipleSingleLineStatementRewrite`
   
   If you would have 3rd level IF/ELSE after  `a[21] = 1;  a[22] = 1;` the story will be the same. My proposition is able to extract such statements for every level. 
   
   And now you may say, that this is not a big deal. Maybe with an example I'm showing here the gain is hard to spot. However I can tell that for a production job (SQL Query), that was causing FLINK-27246, I have an extracted method that has 537 statements similar to those from example below, that I'm not sure if the original implementation could extract. Plus we have many additional methods containing 2 statements. 
   
   So even if this would be the only extra thing (except supporting `else if` and `while`) I think that at scale, the gain adds up.
   
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1418428367

   > After merging this PR I would like to create a back port PRs to 1.15 and 1.16.
   
   You're welcome to create back ports. As 1.16.0 has just released for one month or two I guess 1.16.1 will come in the near future. However 1.15 is an older version and I can't determine when will we release the next version (or if the next version will ever come).


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1067856440


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }

Review Comment:
   It was used at some point during the development, but now it, like you wrote, no one is calling it.
   
   I will remove it.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1070113981


##########
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));

Review Comment:
   This should not be here, I planned to remove this in next commits. 
   I also think that we dont need this block, but maybe I'm missing something.
   With current implementation of we will not have such statement
   ```
   else {
   ...
   }
   ```
   Passed as input to BlockStatementSplitter.
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068106734


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }
+        }
+
+        private boolean canGroupAsSingleStatement(ParserRuleContext context) {
+
+            StatementContext statement;
+
+            if (context instanceof StatementContext) {
+                statement = (StatementContext) context;
+            } else if (context instanceof BlockStatementContext) {
+                statement = ((BlockStatementContext) context).statement();
+            } else {
+                return false;
+            }
+
+            return statement != null
+                    && (statement.IF() != null
+                            || statement.ELSE() != null
+                            || statement.WHILE() != null);
+        }
+
+        private boolean shouldExtract(StatementContext ctx) {
+            return ctx != null
+                    && ctx.block() != null
+                    && ctx.block().blockStatement() != null
+                    // if there is only one statement in the block it's useless to extract
+                    // it into a separate function
+                    && ctx.block().blockStatement().size() > 1
+                    // should not extract blocks with return statements
+                    && getNumReturnsInContext(ctx.block()) == 0;
+        }

Review Comment:
   Good point, thank you.
   
   I have enhanced the `ReturnCounter`, since it was used only for this check to include `break` and `continue`.
   I have extracted it to separate file and renamed to `ReturnAndJumpCounter`



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1384203356

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1066817943


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }

Review Comment:
   OK I understand. The code is not completely rewritten. Only statements in `groups` are replaced and other statements are kept untouched. So this checking is needed.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1370869797

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1374392355

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1411839600

   Thanks @kristoffSC for your effort in this long running pull request. I don't have much to say now. You did a great job.
   
   One last comment: Add a test to `CodeSplitITCase` to address why this issue is brought up. Maciej Bryński's comment in the original JIRA ticket might be useful.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] fapaul commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
fapaul commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1376945309

   I have very limited knowledge with that part of the code base. Can you help with the review here @JingsongLi , @tsreaper ?


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1066708360


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }
+        }
+
+        private boolean canGroupAsSingleStatement(ParserRuleContext context) {
+
+            StatementContext statement;
+
+            if (context instanceof StatementContext) {
+                statement = (StatementContext) context;
+            } else if (context instanceof BlockStatementContext) {
+                statement = ((BlockStatementContext) context).statement();
+            } else {
+                return false;
+            }
+
+            return statement != null
+                    && (statement.IF() != null
+                            || statement.ELSE() != null
+                            || statement.WHILE() != null);
+        }
+
+        private boolean shouldExtract(StatementContext ctx) {
+            return ctx != null
+                    && ctx.block() != null
+                    && ctx.block().blockStatement() != null
+                    // if there is only one statement in the block it's useless to extract
+                    // it into a separate function
+                    && ctx.block().blockStatement().size() > 1
+                    // should not extract blocks with return statements
+                    && getNumReturnsInContext(ctx.block()) == 0;
+        }
+
+        private int getNumReturnsInContext(ParserRuleContext ctx) {
+            ReturnCounter counter = new ReturnCounter();
+            counter.visit(ctx);
+            return counter.returnCount;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> rewriteToGroups() {
+            for (BlockStatementGrouperVisitor child : children) {
+                child.rewriteToGroups();
+            }
+
+            for (Pair<String, List<LocalGroupElement>> group : groups) {
+                List<LocalGroupElement> value = group.getValue();
+                rewriter.replace(
+                        value.get(0).getStart(),
+                        value.get(value.size() - 1).getStop(),
+                        group.getKey() + "(" + this.parameters + ");");
+            }
+
+            return groups;
+        }
+    }
+
+    private interface LocalGroupElement {
+
+        Token getStart();
+
+        Token getStop();
+
+        String getBody();
+
+        ParserRuleContext getContext();
+    }
+
+    private static class ContextGroupElement implements LocalGroupElement {
+
+        private final ParserRuleContext parserRuleContext;
+
+        private ContextGroupElement(ParserRuleContext parserRuleContext) {
+            this.parserRuleContext = parserRuleContext;
+        }
+
+        @Override
+        public Token getStart() {
+            return this.parserRuleContext.start;
+        }
+
+        @Override
+        public Token getStop() {
+            return this.parserRuleContext.stop;
+        }
+
+        @Override
+        public String getBody() {
+            return CodeSplitUtil.getContextString(this.parserRuleContext);
+        }
+
+        @Override
+        public ParserRuleContext getContext() {
+            return this.parserRuleContext;
+        }
+    }
+
+    private static class RewriteContextGroupElement implements LocalGroupElement {
+
+        private final ParserRuleContext parserRuleContext;
+
+        private final TokenStreamRewriter rewriter;
+
+        private RewriteContextGroupElement(
+                ParserRuleContext parserRuleContext, TokenStreamRewriter rewriter) {
+            this.parserRuleContext = parserRuleContext;
+            this.rewriter = rewriter;
+        }
+
+        @Override
+        public Token getStart() {
+            return this.parserRuleContext.start;
+        }
+
+        @Override
+        public Token getStop() {
+            return this.parserRuleContext.stop;
+        }
+
+        @Override
+        public String getBody() {
+            return this.rewriter.getText();
+        }
+
+        @Override
+        public ParserRuleContext getContext() {
+            return this.parserRuleContext;
+        }
+    }
+
+    private static class ReturnCounter extends JavaParserBaseVisitor<Void> {
+
+        private int returnCount = 0;
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+            if (ctx.RETURN() != null) {
+                returnCount++;
+            }
+            return visitChildren(ctx);
+        }
+    }
+
+    /**
+     * This object represents a rewritten code block. It contains its new form along with all
+     * extracted groups and their names.
+     */
+    public static class RewriteGroupedCode {
+
+        /** Rewritten code block containing calls to extracted methods. */
+        private final String rewriteCode;
+
+        /** All extracted groups with their names. */
+        private final List<Pair<String, List<String>>> groups;

Review Comment:
   `Pair` -> `Tuple2`. `Tuple2` is a Flink built-in class.



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and

Review Comment:
   What are "end extract single line statements"? Could you explain more?



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }

Review Comment:
   What's the usage of this method? No one is calling it.



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();

Review Comment:
   ```suggestion
                               localGroupCodeLength = 0;
                           }
                           localGroupCodeLength += bsc.getText().length();
   ```



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }
+        }
+
+        private boolean canGroupAsSingleStatement(ParserRuleContext context) {
+
+            StatementContext statement;
+
+            if (context instanceof StatementContext) {
+                statement = (StatementContext) context;
+            } else if (context instanceof BlockStatementContext) {
+                statement = ((BlockStatementContext) context).statement();
+            } else {
+                return false;
+            }
+
+            return statement != null
+                    && (statement.IF() != null
+                            || statement.ELSE() != null
+                            || statement.WHILE() != null);
+        }
+
+        private boolean shouldExtract(StatementContext ctx) {
+            return ctx != null
+                    && ctx.block() != null
+                    && ctx.block().blockStatement() != null
+                    // if there is only one statement in the block it's useless to extract
+                    // it into a separate function
+                    && ctx.block().blockStatement().size() > 1
+                    // should not extract blocks with return statements
+                    && getNumReturnsInContext(ctx.block()) == 0;
+        }

Review Comment:
   What about `break`s and `continue`s in a `while` loop? You can't partition a `while` loop into different methods if it has `break`s or `continue`s.



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }

Review Comment:
   Let's say each group will contain 5 statements. If there are 6 statements in the current method, this `if` checking will skip the last statement and it will be lost.
   
   Add a test to cover the corner case.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1372137159

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068871408


##########
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));

Review Comment:
   Remove `System.out`. Maybe assert that this block should never be executed?



##########
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:
   Why are you adding blocks to the recursively created `BlockStatementVisitor`? This will cause the problem that blocks in `extractedSingleBlocks` might not be the same block.



##########
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);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {

Review Comment:
   Not all statements are added to `extractedSingleBlocks`. In that case `extractedSingleBlocks` is not a continuous block of code.
   
   In `rewrite` you replace everything from the first element of `extractedSingleBlocks` to the last element. I guess this implies that `extractedSingleBlocks` is continuous. Why can you do this?



##########
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(

Review Comment:
   Why do you need to `mergeBlocks`? Statements from different levels of blocks may be extracted into the same new method?



##########
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);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }

Review Comment:
   Why not just recursively call `BlockStatementVisitor` without branching? Let the newly created `BlockStatementVisitor` handle `WHILE`s and `IF`s.
   
   Specialized `bscCounter` is also not needed. Just use `counter`.
   
   ```suggestion
                   if (ctx.block() != null) {
                       for (BlockStatementContext bsc : ctx.block().blockStatement()) {
                           String localContext =
                                   String.format("%s_%d", this.context, counter++);
                           BlockStatementVisitor visitor =
                                   new BlockStatementVisitor(rewriter, localContext, parameters);
                           visitor.addBlocks(extractedSingleBlocks);
   
                           extractedSingleBlocks.clear();
                           children.add(visitor);
                           visitor.visitStatement(bsc.statement());
                       }
                   } else {
                       if (shouldExtract(ctx)) {
                           extractedSingleBlocks.add(ctx);
                       }
                   }
   ```



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071317204


##########
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);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }

Review Comment:
   Your proposition will produce overall correct result but with mu current implementation the outcome has slightly better.
   
   Given:
   ```
   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];
           }
       }
   ```
   
   Yours will produce:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
     
     if (a[0] == 0) {
     myFun1_rewriteGroup1(a, b);
     
     myFun1_rewriteGroup2(a, b);
     } else {
     myFun1_rewriteGroup3(a, b);
     a[2] = b[2];
     }
     }
     
     void myFun1_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
     a[11] = b[0];
     a[12] = b[0];
     if (a[2] == 0) {
     myFun1_rewriteGroup0_rewriteGroup0(a, b);
     } else {
     myFun1_rewriteGroup0_rewriteGroup1(a, b);
     }
     }
     
     void myFun1_rewriteGroup3(int[] a, int[] b) throws RuntimeException {
     a[0] = b[0];
     a[1] = b[1];
     }
     
     void myFun1_rewriteGroup2(int[] a, int[] b) throws RuntimeException {
     a[13] = b[0];
     a[14] = b[0];
     }
     
     void myFun1_rewriteGroup0_rewriteGroup0(int[] a, int[] b) throws RuntimeException {
     a[21] = 1;
     a[22] = 1;
     }
     
     void myFun1_rewriteGroup0_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
     a[23] = b[2];
     a[24] = b[2];
     }
   ```
   
   And mine will give us 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];
     }
   ```
   
   The differnece is that in my proposition statements ` a[11] = b[0];` and `a[12] = b[0];` are extracted to separate method and block
   ```
               a[0] = b[0];
               a[1] = b[1];
               a[2] = b[2];
   ```
   Is extracted as whole to new method, where in yours, the last statement was not included.
   I can imagine that for more complex statement, this difference can add up.
   
   Please see my other replays about this logic for more details. 
   
   For Specialized bscCounter, you are right. I will remove it.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071174072


##########
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(

Review Comment:
   The level of block is identified by its own "context" value. So The context value become automatically the name of extracted method.
   Blocks with the same "context" should be extracted to the same method.
   
   In here I could use `map.putAll` but then there would be a risk to override values for already existing key (context).
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
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: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1419110913

   btw, i created an issue for that
   https://issues.apache.org/jira/browse/FLINK-30927


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071233842


##########
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);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {

Review Comment:
   HI @tsreaper,
   so what maybe was not fully clear is that `extractedSingleBlocks` is a continuous block of single line statements up to next `block` expression so up to next IF/ELSE/WHILE block.
   
   Lets break down below example:
   Having:
   ```
   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[2] = 1;
               } else {
                   a[2] = b[2];
               }
   
               a[13] = b[0];
               a[14] = b[0];
           } else {
               a[0] = b[0];
               a[1] = b[1];
               a[2] = b[2];
           }
    }
   ```
   
   With current implementation will be rewrite to:
     ```
   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_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
   	myFun1_ifBody0_0(a, b);
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
     }
     
     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];
     }
   ```
   
   The provided example has "non continuous block" of single line statements in `TRUE` branch of first level IF/ELSE block. The block is spitted by nested, 2nd level IF/ELSE block.
   
   The first half of the single line statement block (a[11], a[12]) were grouped and extracted to new method together with 2nd level if/esle block - `myFun1_rewriteGroup1()`.In there we can see that those single line statements were extracted further to new method -> `myFun1_ifBody0_0`
   
   The second half, expression a[13] and a[14] were extracted to their own method -> `myFun1_ifBody0`
   
   While processing a statements, for example `TRUE` branch, whenever we will spot a new IF or ELSE we will "copy all single line statements" to newly created Visitor. The `extractedSingleBlocks` array for the "original visitor" will be cleared and ready to accept new single line statements. 
   
   This is exactly what happen with a[11], a[12] from example above. They were added to the new Visitor created for 2nd level IF/ELSE. The a[13] and a[14] expressions were were added to cleared `extractedSingleBlocks` array in original Visitor starting a new continuous block.
   
   Also maybe refer to this comment -> where I explained the high level logic https://github.com/apache/flink/pull/21393#discussion_r1068750626
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1081158207


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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;
+    }

Review Comment:
   So this method only returns a map with 1 pair of key-value, and the key is known to the caller (which is `context` of `extractBlocks`). Why not just return a string `visitor.rewriter.getText()`?



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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.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, counter++);
+                            BlockStatementVisitor visitor =
+                                    new BlockStatementVisitor(rewriter, localContext, parameters);
+                            visitor.addBlocks(extractedSingleBlocks);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }
+            }
+            return null;
+        }
+
+        public void rewrite() {
+
+            for (BlockStatementVisitor child : children) {
+                child.rewrite();
+            }
+
+            // if there is only one statement in the block, and it is not IF/ELSE/WHILE statement
+            // it's pointless to extract it into a separate function.
+            if (extractedSingleBlocks.size() > 1
+                    || (extractedSingleBlocks.size() == 1
+                            && canGroupAsSingleStatement(extractedSingleBlocks.get(0)))) {
+                rewriter.replace(
+                        extractedSingleBlocks.get(0).start,
+                        extractedSingleBlocks.get(extractedSingleBlocks.size() - 1).stop,
+                        context + "(" + parameters + ");");
+            }
+        }
+
+        public Map<String, List<String>> getAllBlocks() {
+            Map<String, List<String>> blocks = new HashMap<>();
+            for (BlockStatementVisitor child : children) {
+                Map<String, List<String>> rewrite = child.getAllBlocks();
+                mergeBlocks(rewrite, blocks);
+            }
+
+            List<String> myBlocks = new ArrayList<>();
+            for (ParserRuleContext block : extractedSingleBlocks) {
+                myBlocks.add(CodeSplitUtil.getContextString(block));
+            }
+
+            // Rewrite only those blocks that have more than one statement in it.
+            if (myBlocks.size() > 1) {
+                mergeBlocks(Collections.singletonMap(context, myBlocks), blocks);
+            }
+            return blocks;
+        }
+
+        private void addBlocks(List<ParserRuleContext> blocks) {

Review Comment:
   > addBlocks -> removed
   
   It's still here.



##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   > recursion -> I really don't think that this should be a problem. Recursion is called only to gather the results, parsing is not using recursion.
   
   I didn't mean performance. It's just that, this class is sort of difficult for me to understand without proper comments. I'm afraid other readers will feel the same.
   
   By "you can implement this with just one for loop and a recursive call" I mean something like the following.
   
   ```java
   private final Map<String, List<ParserRuleContext>> blocks = new HashMap<>();
   
   public void visitBlock(JavaParser.BlockContext ctx, String context) {
       int counter = 0;
       boolean shouldExtract = shouldExtract(ctx);
       List<ParserRuleContext> extractedSingleBlocks = new ArrayList<>();
   
       for (BlockStatementContext bsc : ctx.blockStatement()) {
           if (bsc.statement() != null
                   && bsc.statement().block() != null
                   && (bsc.statement().IF() != null
                           || bsc.statement().ELSE() != null
                           || bsc.statement().WHILE() != null)) {
               blocks.put(context, extractedSingleBlocks);
               extractedSingleBlocks = new ArrayList<>();
   
               String localContext = String.format("%s_%d", context, counter++);
               visitBlock(bsc.statement().block(), localContext);
           } else {
               if (shouldExtract) {
                   extractedSingleBlocks.add(bsc);
               }
           }
       }
   }
   ```
   
   You'll call `visitBlock(methodBody().block(), methodName)` to start the recursive process.
   
   This code might not run as expected but I hope it illustrates my idea. My main goal here is to remove `mergeBlocks` because it's very confusing to understand without proper comments. Readers may think that you're grouping single statements from different levels, but actually you're not.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1081163322


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   > recursion -> I really don't think that this should be a problem. Recursion is called only to gather the results, parsing is not using recursion.
   
   I didn't mean performance. It's just that, this class is sort of difficult for me to understand without proper comments. I'm afraid other readers will feel the same.
   
   By "you can implement this with just one for loop and a recursive call" I mean something like the following.
   
   ```java
   private final Map<String, List<ParserRuleContext>> blocks = new HashMap<>();
   
   public void visitBlock(JavaParser.BlockContext ctx, String context) {
       int counter = 0;
       boolean shouldExtract = shouldExtract(ctx);
       List<ParserRuleContext> extractedSingleBlocks = new ArrayList<>();
   
       for (BlockStatementContext bsc : ctx.blockStatement()) {
           if (bsc.statement() != null
                   && bsc.statement().block() != null
                   && (bsc.statement().IF() != null
                           || bsc.statement().ELSE() != null
                           || bsc.statement().WHILE() != null)) {
               blocks.put(context, extractedSingleBlocks);
               extractedSingleBlocks = new ArrayList<>();
   
               String localContext = String.format("%s_%d", context, counter++);
               visitBlock(bsc.statement().block(), localContext);
           } else {
               if (shouldExtract) {
                   extractedSingleBlocks.add(bsc);
               }
           }
       }
   }
   ```
   
   You'll call `visitBlock(methodBody().block(), methodName)` to start the recursive process.
   
   This code might not run as expected but I hope it illustrates my idea. My main goal here is to remove `addBlocks` and `mergeBlocks` because it's very confusing to understand without proper comments. Readers may think that you're grouping single statements from different levels, but actually you're not.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1081192318


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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.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, counter++);
+                            BlockStatementVisitor visitor =
+                                    new BlockStatementVisitor(rewriter, localContext, parameters);
+                            visitor.addBlocks(extractedSingleBlocks);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }
+            }
+            return null;
+        }
+
+        public void rewrite() {
+
+            for (BlockStatementVisitor child : children) {
+                child.rewrite();
+            }
+
+            // if there is only one statement in the block, and it is not IF/ELSE/WHILE statement
+            // it's pointless to extract it into a separate function.
+            if (extractedSingleBlocks.size() > 1
+                    || (extractedSingleBlocks.size() == 1
+                            && canGroupAsSingleStatement(extractedSingleBlocks.get(0)))) {
+                rewriter.replace(
+                        extractedSingleBlocks.get(0).start,
+                        extractedSingleBlocks.get(extractedSingleBlocks.size() - 1).stop,
+                        context + "(" + parameters + ");");
+            }
+        }
+
+        public Map<String, List<String>> getAllBlocks() {
+            Map<String, List<String>> blocks = new HashMap<>();
+            for (BlockStatementVisitor child : children) {
+                Map<String, List<String>> rewrite = child.getAllBlocks();
+                mergeBlocks(rewrite, blocks);
+            }
+
+            List<String> myBlocks = new ArrayList<>();
+            for (ParserRuleContext block : extractedSingleBlocks) {
+                myBlocks.add(CodeSplitUtil.getContextString(block));
+            }
+
+            // Rewrite only those blocks that have more than one statement in it.
+            if (myBlocks.size() > 1) {
+                mergeBlocks(Collections.singletonMap(context, myBlocks), blocks);
+            }
+            return blocks;
+        }
+
+        private void addBlocks(List<ParserRuleContext> blocks) {

Review Comment:
   yes, my bad, I meant `addGroups `
   You were referring to `addGroups ` previously.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper merged pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper merged PR #21393:
URL: https://github.com/apache/flink/pull/21393


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1086577308


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   I understand. If it is difficult to refactor this part of code, at least add detailed comments showing that we're only grouping single statements on the same level, not merging different levels together. In `mergeBlocks` you should also assert that two maps do not have the same key.
   
   There is still one comment on `rewriteBlock`, please take a look.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1405572025

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1371110633

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1372082126

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1372019046

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1367883271

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068750626


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);
+
+                        mergeBlocks(rewrite, extractedBlocks);
+                    }
+                }
+            }
+        }
+
+        Map<String, List<String>> localBlocks = visitor.getAllBlocks();
+        mergeBlocks(localBlocks, extractedBlocks);
+
+        return extractedBlocks;
+    }
+
+    /**
+     * 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() {
+
+        for (BlockStatementSplitter child : children) {
+            child.visitor.rewrite();
+        }
+        visitor.rewrite();
+
+        Map<String, String> rewriteBlocks = new HashMap<>();
+        for (BlockStatementSplitter child : children) {
+            Map<String, String> childRewriteBlocks = child.rewriteBlock();
+            for (Entry<String, String> entry : childRewriteBlocks.entrySet()) {
+                rewriteBlocks.merge(
+                        entry.getKey(),
+                        entry.getValue(),
+                        (s, s2) -> {
+                            throw new RuntimeException(
+                                    String.format(
+                                            "Override rewritten block  for key %s. Blocks are %s -> %s",
+                                            entry.getKey(), s, s2));
+                        });
+            }
+        }
+
+        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<ContextTextPair> 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 {

Review Comment:
   Hi @tsreaper 
   In fact we don't need `ctx.ELSE()` branch at all. I will try to explain why.
   
   The initial input statement for `BlockStatementSplitter` can be two things:
   
   - a statement that is represented like this:
   
   ```
   if (a[2] == 0) {
       a[2] = 1;
   } else {
       a[2] = b[2];
   }
   ```
   
   - a block, that will be represented as this:
   
   ```
   {
       if (a[2] == 0) {
           a[2] = 1;
       } else {
   	a[2] = b[2];
       }
   }
   ```
   
   Mind that block has additional brackets `{ }` at the beginning and at the end. 
   
   When `BlockStatementSplitter` is called from `BlockStatementRewriter` it will always be called by passing the statement, so without brackets `{ }`. The statement will never start from `else`, it will start from IF or WHILE (this is what we detect for now) or it will be what I call a "single line statement", meaning operation on a field, for example `a[2] = 1;`.
   
   Lets look at the example. The `BlockStatementRewriter` called `BlockStatementSplitter::visitStatement` with below statement:
   
   ```
   if (a[0] == 0) {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   } else {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   
   For this, the condition `if (ctx.WHILE() != null)` will be true. 
   In this branch we will iterate through every `ctx.statement()`
   The first item would be:
   
   ```
   {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   }
   ```
   So the `TRUE` branch of the first level IF/ELSE expression. We will create a new context to indicate that we will process an IF(possibly also ELSE) statement and we will create new `BlockStatementVisitor` and call visitStatement method to process that block, since it can contain more nested structures we would like to rewrite. Mind that now the input argument is a "block", it starts and ends with brackets `{ }`
   
   The second item in the iteration routine will be this:
   ```
   {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   So the `FALSE` branch of the first level IF/ELSE expression. Same as for TRUE, we will create new context and `BlockStatementVisitor` to process that block since it can contain more nested structures we would like to rewrite. In this case the input also would be the "block" since it starts and ends with brackets `{ }`, same as for TRUE branch.
   
   In both new Visitors, conditions `ctx.WHILE() != null` and `ctx.IF() != null` will be false since now we have a "block", it starts from `{`. So we will land at `if (ctx.block() != null) ` line ~224 of `BlockStatementSplitter.java`. Since this is a block now, we need to iterate over all its statements hence `for (BlockStatementContext bsc : ctx.block().blockStatement())`
   
   For `TRUE` branch from the initial statement it will be only one item:
   ```
   if (a[2] == 0) {
   	a[2] = 1;
   } else {
   	a[2] = b[2];
   }
   ```
   We will create new Visitor and process this statement further.
   
   For `FALSE` branch, from the initial statement it will contain two elements. Those will be "single line elements":
   - a[0] = b[0];
   - a[1] = b[1];
   
   Those will be added to `extractedSingleBlocks` blocks and extracted to separate method.
   
   I hope that I managed to explain the logic here, let me know please.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1081192318


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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.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, counter++);
+                            BlockStatementVisitor visitor =
+                                    new BlockStatementVisitor(rewriter, localContext, parameters);
+                            visitor.addBlocks(extractedSingleBlocks);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }
+            }
+            return null;
+        }
+
+        public void rewrite() {
+
+            for (BlockStatementVisitor child : children) {
+                child.rewrite();
+            }
+
+            // if there is only one statement in the block, and it is not IF/ELSE/WHILE statement
+            // it's pointless to extract it into a separate function.
+            if (extractedSingleBlocks.size() > 1
+                    || (extractedSingleBlocks.size() == 1
+                            && canGroupAsSingleStatement(extractedSingleBlocks.get(0)))) {
+                rewriter.replace(
+                        extractedSingleBlocks.get(0).start,
+                        extractedSingleBlocks.get(extractedSingleBlocks.size() - 1).stop,
+                        context + "(" + parameters + ");");
+            }
+        }
+
+        public Map<String, List<String>> getAllBlocks() {
+            Map<String, List<String>> blocks = new HashMap<>();
+            for (BlockStatementVisitor child : children) {
+                Map<String, List<String>> rewrite = child.getAllBlocks();
+                mergeBlocks(rewrite, blocks);
+            }
+
+            List<String> myBlocks = new ArrayList<>();
+            for (ParserRuleContext block : extractedSingleBlocks) {
+                myBlocks.add(CodeSplitUtil.getContextString(block));
+            }
+
+            // Rewrite only those blocks that have more than one statement in it.
+            if (myBlocks.size() > 1) {
+                mergeBlocks(Collections.singletonMap(context, myBlocks), blocks);
+            }
+            return blocks;
+        }
+
+        private void addBlocks(List<ParserRuleContext> blocks) {

Review Comment:
   yes, my bad, I meant `addGroups `



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1419079063

   somehow build doesn't fail
   however lots of output in tests with generated code
   The output happens here https://github.com/apache/flink/blob/2ae5df278958073fee63b2bf824a53a28a21701b/flink-table/flink-table-runtime/src/main/java/org/apache/flink/table/runtime/generated/CompileUtils.java#L106
   
   looks like the exception is caught somewhere and not rethrown further
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] snuyanzin commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "snuyanzin (via GitHub)" <gi...@apache.org>.
snuyanzin commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1419057945

   it looks like it generates methods with same signature for some cases e.g. ` org.apache.flink.table.planner.runtime.stream.sql.MatchRecognizeITCase#testUserDefinedFunctions`
   and it leads to debug output for these tests


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1419069671

   @snuyanzin
   
   > it looks like it generates methods with same signature for some cases e.g.
    I will take a look at this one. 
    
    This was not causing CI build to fail?


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1090568909


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables, IF and
+ * WHILE statements and extract new method for each group making them smaller.
+ *
+ * <p>BlockStatementGrouper 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>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *     myFun_rewriteGroup4(a, b);
+ *     myFun_rewriteGroup5(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup4 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_1_rewriteGroup3(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup5 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_1_rewriteGroup3 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(maxMethodLength, parameters);
+        CommonTokenStream tokenStream =
+                new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+        JavaParser javaParser = new JavaParser(tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        TokenStreamRewriter rewriter = new TokenStreamRewriter(tokenStream);
+        visitor.visitStatement(javaParser.statement(), context, rewriter);
+
+        visitor.rewrite();
+        Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups = visitor.groups;
+
+        Map<String, List<String>> groupStrings = new HashMap<>(groups.size());
+        for (Entry<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> group :
+                groups.entrySet()) {
+            List<String> collectedStringGroups =
+                    group.getValue().getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+
+            groupStrings.put(group.getKey(), collectedStringGroups);
+        }
+
+        return new RewriteGroupedCode(rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor {
+
+        private final Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups =
+                new HashMap<>();
+
+        private final long maxMethodLength;
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(long maxMethodLength, String parameters) {
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        public void visitStatement(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+
+            if (ctx.getChildCount() == 0) {
+                return;
+            }
+
+            // For these statements here we want to process all "branches" separately, for example
+            // TRUE and FALSE branch of IF/ELSE block.
+            // each statement can be rewritten and extracted.
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        String localContext = String.format("%s_%d", context, counter++);
+                        groupBlock(statement, localContext, rewriter);
+                    }
+                }
+            } else {
+                // The block did not start from IF/ELSE/WHILE statement
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx, context, rewriter);
+                }
+            }
+        }
+
+        // Group continuous block of statements together. If Statement is an IF/ELSE/WHILE,
+        // its body can be further grouped by recursive call to visitStatement method.
+        private void groupBlock(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + this.counter++;
+
+                    CommonTokenStream tokenStream =
+                            new CommonTokenStream(
+                                    new JavaLexer(
+                                            CharStreams.fromString(
+                                                    CodeSplitUtil.getContextString(statement))));

Review Comment:
   Please remember that that comment was created for previous implementation, where I was using multiple instances of Splitter and Grouper visitors. I refactor both classes so now they are using recursive calls to `visit...` methods. 
   
   Regarding the "repeatedly parse" part.
   Statement is processed by Splitter and Grouper once. I'm not sure how it looks from the Antlr4 but I guess it will be lie so:
   1. Initial parse for Splitter (part of original block statement to rewrite)
   2. Initial parser for Grouper  (part of original block statement to rewrite)
   3. possibly a third time when we create a new rewriter instane for `RewriteContextGroupElement`
   
   I think that my main point was that in my implementation I don't parse entire class again every time when `IfStatementRewriter` is doing another round to process extracted methods. At least this is what I think is happening in `IfStatementRewriter` and that was the reason you had
   
   ```
           while (visitor.hasRewrite()) {
               visitor = new IfStatementVisitor(rewriterCode);
               rewriterCode = visitor.rewriteAndGetCode();
           }
   ```
   
   I think that in my case, the number of times that statement will be parsed by antlr or processed by Splitter/Grouper is constant, whereas in `IfStatementRewriter` it depends on the number of extracted methods.
   
   But I want to be crystal clear here. I dont want to provoke a discussion about the performance especially when you said that with original approach you never had issue regarding the performance - I get that. When I started woring on my solution, the "performance" was not my goal. If there is a performance gain Its a side effect of the design approach I took i think. 
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068750626


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);
+
+                        mergeBlocks(rewrite, extractedBlocks);
+                    }
+                }
+            }
+        }
+
+        Map<String, List<String>> localBlocks = visitor.getAllBlocks();
+        mergeBlocks(localBlocks, extractedBlocks);
+
+        return extractedBlocks;
+    }
+
+    /**
+     * 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() {
+
+        for (BlockStatementSplitter child : children) {
+            child.visitor.rewrite();
+        }
+        visitor.rewrite();
+
+        Map<String, String> rewriteBlocks = new HashMap<>();
+        for (BlockStatementSplitter child : children) {
+            Map<String, String> childRewriteBlocks = child.rewriteBlock();
+            for (Entry<String, String> entry : childRewriteBlocks.entrySet()) {
+                rewriteBlocks.merge(
+                        entry.getKey(),
+                        entry.getValue(),
+                        (s, s2) -> {
+                            throw new RuntimeException(
+                                    String.format(
+                                            "Override rewritten block  for key %s. Blocks are %s -> %s",
+                                            entry.getKey(), s, s2));
+                        });
+            }
+        }
+
+        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<ContextTextPair> 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 {

Review Comment:
   Hi @tsreaper 
   In fact we don't need `ctx.ELSE()` branch at all. I will try to explain why.
   
   The initial input statement for `BlockStatementSplitter` can be two things:
   
   - a statement that is represented like this:
   
   ```
   if (a[2] == 0) {
       a[2] = 1;
   } else {
       a[2] = b[2];
   }
   ```
   
   - a block, that will be represented as this:
   
   ```
   {
       if (a[2] == 0) {
           a[2] = 1;
       } else {
   	a[2] = b[2];
       }
   }
   ```
   
   Mind that block has additional brackets `{ }` at the beginning and at the end. 
   
   When `BlockStatementSplitter` is called from `BlockStatementRewriter` it will always be called by passing the statement, so without brackets `{ }`. The statement will never start from `else`, it will start from IF or WHILE (this is what we detect for now) or it will be what I call a "single line statement", meaning operation on a field, for example `a[2] = 1;`.
   
   Lets look at the example. The `BlockStatementRewriter` called `BlockStatementSplitter::visitStatement` with below statement:
   
   ```
   if (a[0] == 0) {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   } else {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   
   For this, the condition `if (ctx.WHILE() != null)` will be true. 
   In this branch we will iterate through every `ctx.statement()`
   The first item would be:
   
   ```
   {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   }
   ```
   So the `TRUE` branch of the first level IF/ELSE expression. We will create a new context to indicate that we will process an IF(possibly also ELSE) statement and we will create new `BlockStatementVisitor` and call visitStatement method to process that block, since it can contain more nested structures we would like to rewrite. Mind that now the input argument is a "block", it starts and ends with brackets `{ }`
   
   The second item in the iteration routine will be this:
   ```
   {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   So the `FALSE` branch of the first level IF/ELSE expression. Same as for TRUE, we will create new context and `BlockStatementVisitor` to process that block since it can contain more nested structures we would like to rewrite. In this case the input also would be the "block" since it starts and ends with brackets `{ }`, same as for TRUE branch.
   
   In both new Visitors, conditions `ctx.WHILE() != null` and `ctx.IF() != null` will be false since now we have a "block", it starts from `{`. So we will land at `if (ctx.block() != null) ` line ~224 of `BlockStatementSplitter.java`. Since this is a block now, we need to iterate over all its statements hence `for (BlockStatementContext bsc : ctx.block().blockStatement())`
   
   For `TRUE` branch from the initial statement it will be only one item:
   ```
   if (a[2] == 0) {
   	a[2] = 1;
   } else {
   	a[2] = b[2];
   }
   ```
   We will create new Visitor and process this statement further.
   
   For `FALSE` branch, from the initial statement it will contain two elements. Those will be "single line elements":
   - a[0] = b[0];
   - a[1] = b[1];
   
   Those will be added to `extractedSingleBlocks` blocks and extracted to separate method.
   
   I hope that clear the logic a little bit.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1067856440


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }

Review Comment:
   It was used at some point during the development, but now like you wrote, no one is calling it.
   
   I will remove it.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071317204


##########
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);
+
+                            extractedSingleBlocks.clear();
+                            children.add(visitor);
+                            visitor.visitStatement(bsc.statement());
+                        } else {
+                            if (shouldExtract(ctx)) {
+                                extractedSingleBlocks.add(bsc);
+                            }
+                        }
+                    }
+                } else {
+                    extractedSingleBlocks.add(ctx);
+                }

Review Comment:
   Your proposition will produce overall correct result but with mu current implementation the outcome has slightly better.
   
   Given:
   ```
   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];
           }
       }
   ```
   
   Yours will produce:
   ```
   public void myFun1(int[] a, int[] b) throws RuntimeException {
     
     if (a[0] == 0) {
     myFun1_rewriteGroup1(a, b);
     
     myFun1_rewriteGroup2(a, b);
     } else {
     myFun1_rewriteGroup3(a, b);
     a[2] = b[2];
     }
     }
     
     void myFun1_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
     a[11] = b[0];
     a[12] = b[0];
     if (a[2] == 0) {
     myFun1_rewriteGroup0_rewriteGroup0(a, b);
     } else {
     myFun1_rewriteGroup0_rewriteGroup1(a, b);
     }
     }
     
     void myFun1_rewriteGroup3(int[] a, int[] b) throws RuntimeException {
     a[0] = b[0];
     a[1] = b[1];
     }
     
     void myFun1_rewriteGroup2(int[] a, int[] b) throws RuntimeException {
     a[13] = b[0];
     a[14] = b[0];
     }
     
     void myFun1_rewriteGroup0_rewriteGroup0(int[] a, int[] b) throws RuntimeException {
     a[21] = 1;
     a[22] = 1;
     }
     
     void myFun1_rewriteGroup0_rewriteGroup1(int[] a, int[] b) throws RuntimeException {
     a[23] = b[2];
     a[24] = b[2];
     }
   ```
   
   And mine will give us 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];
     }
   ```
   
   The differnece is that in my proposition statements ` a[11] = b[0];` and `a[12] = b[0];` are extracted to separate method and block
   ```
               a[0] = b[0];
               a[1] = b[1];
               a[2] = b[2];
   ```
   Is extracted as whole to new method, where in yours, the last statement was not included.
   I can imagine that for more complex statement, this difference can add up.
   
   For Specialized bscCounter, you are right. I will remove it.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071557132


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();
+                        localGroup.add(new ContextGroupElement(bsc));
+                    }
+                }
+            }
+
+            if (localGroup.size() > 1
+                    || (localGroup.size() == 1
+                            && canGroupAsSingleStatement(localGroup.get(0).getContext()))) {
+                String localContext = context + "_rewriteGroup" + localCounter;
+                groups.add(Pair.of(localContext, localGroup));
+            }
+        }
+
+        private boolean canGroupAsSingleStatement(ParserRuleContext context) {
+
+            StatementContext statement;
+
+            if (context instanceof StatementContext) {
+                statement = (StatementContext) context;
+            } else if (context instanceof BlockStatementContext) {
+                statement = ((BlockStatementContext) context).statement();
+            } else {
+                return false;
+            }
+
+            return statement != null
+                    && (statement.IF() != null
+                            || statement.ELSE() != null
+                            || statement.WHILE() != null);
+        }
+
+        private boolean shouldExtract(StatementContext ctx) {
+            return ctx != null
+                    && ctx.block() != null
+                    && ctx.block().blockStatement() != null
+                    // if there is only one statement in the block it's useless to extract
+                    // it into a separate function
+                    && ctx.block().blockStatement().size() > 1
+                    // should not extract blocks with return statements
+                    && getNumReturnsInContext(ctx.block()) == 0;
+        }
+
+        private int getNumReturnsInContext(ParserRuleContext ctx) {
+            ReturnCounter counter = new ReturnCounter();
+            counter.visit(ctx);
+            return counter.returnCount;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> rewriteToGroups() {
+            for (BlockStatementGrouperVisitor child : children) {
+                child.rewriteToGroups();
+            }
+
+            for (Pair<String, List<LocalGroupElement>> group : groups) {
+                List<LocalGroupElement> value = group.getValue();
+                rewriter.replace(
+                        value.get(0).getStart(),
+                        value.get(value.size() - 1).getStop(),
+                        group.getKey() + "(" + this.parameters + ");");
+            }
+
+            return groups;
+        }
+    }
+
+    private interface LocalGroupElement {
+
+        Token getStart();
+
+        Token getStop();
+
+        String getBody();
+
+        ParserRuleContext getContext();
+    }
+
+    private static class ContextGroupElement implements LocalGroupElement {
+
+        private final ParserRuleContext parserRuleContext;
+
+        private ContextGroupElement(ParserRuleContext parserRuleContext) {
+            this.parserRuleContext = parserRuleContext;
+        }
+
+        @Override
+        public Token getStart() {
+            return this.parserRuleContext.start;
+        }
+
+        @Override
+        public Token getStop() {
+            return this.parserRuleContext.stop;
+        }
+
+        @Override
+        public String getBody() {
+            return CodeSplitUtil.getContextString(this.parserRuleContext);
+        }
+
+        @Override
+        public ParserRuleContext getContext() {
+            return this.parserRuleContext;
+        }
+    }
+
+    private static class RewriteContextGroupElement implements LocalGroupElement {
+
+        private final ParserRuleContext parserRuleContext;
+
+        private final TokenStreamRewriter rewriter;
+
+        private RewriteContextGroupElement(
+                ParserRuleContext parserRuleContext, TokenStreamRewriter rewriter) {
+            this.parserRuleContext = parserRuleContext;
+            this.rewriter = rewriter;
+        }
+
+        @Override
+        public Token getStart() {
+            return this.parserRuleContext.start;
+        }
+
+        @Override
+        public Token getStop() {
+            return this.parserRuleContext.stop;
+        }
+
+        @Override
+        public String getBody() {
+            return this.rewriter.getText();
+        }
+
+        @Override
+        public ParserRuleContext getContext() {
+            return this.parserRuleContext;
+        }
+    }
+
+    private static class ReturnCounter extends JavaParserBaseVisitor<Void> {
+
+        private int returnCount = 0;
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+            if (ctx.RETURN() != null) {
+                returnCount++;
+            }
+            return visitChildren(ctx);
+        }
+    }
+
+    /**
+     * This object represents a rewritten code block. It contains its new form along with all
+     * extracted groups and their names.
+     */
+    public static class RewriteGroupedCode {
+
+        /** Rewritten code block containing calls to extracted methods. */
+        private final String rewriteCode;
+
+        /** All extracted groups with their names. */
+        private final List<Pair<String, List<String>>> groups;

Review Comment:
   Hi,
   I always thought about Flink's  Tuple classes as designed to use in Flink's DataStream Applications rather than core/internal implementations. 
   
   I did a quick search in Flink's base code and I see that `org.apache.commons.lang3.tuple.Pair` is used in quite few places like:
   Hive-connector
   CoGroupOperator (flink-java)
   Few classes from Java-runtime module
   Flink-table module
   
   That's why I considered it as well suited class since it is already on a class path.
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068067248


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and

Review Comment:
   The single line statements are those:
   
   > b[counter] = a[counter] * 3;
   > System.out.println(b[counter]);
   
   Such continous statements, if they are in IF/ELSE or WHILE body, willy be grouped and extracted to separate 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068628274


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);

Review Comment:
   Actually the `BlockStatementSplitter#extractBlocks` and `BlockStatementSplitter`'s `children` array was not needed. I have removed it and now it seems more straight forward.
   
   I will add additional javadoc explaining logic in more 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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1085531144


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   @tsreaper 
   Sorry for delay on this one.
   
   Regarding:
   > I mean something like the following.
   
   I understand your suggestion, I was thinking about similar thing at the beginning of designing the solution.
   
   Let me try to rephrase what I think is main difference in yours and mine proposition.
   
   You are proposing to have single Visitor instance that will recursively parse every statement/block.  Every statement extracted from particular "level" will be added to `blocks` map, where key is "context" that identifies the "level/block" that is currently processed.
   
   In my proposition, I'm creating new Visitor for every new nested "block/level". Every processor maintains its own list of extracted statements. At the end I'm traversing through all visitors and gather all extracted blocks -> mergeBlock method (not the best name I agree).
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1419116469

   I already have fix, will provide PR shortly.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1405755647

   @tsreaper 
   I've refactored both `BlockStatementSplitter` and `BlockStatementGrouper` so they use one `for` loop and one `visitor` instance like you suggested in your comments. There is no merge/add blocks now.
   
   I would appreciate if you take a look and let me know what do you think.
   
   Thanks.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1372780716

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1081214977


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   >  My main goal here is to remove addBlocks and mergeBlocks because it's very confusing to understand without proper comments. Readers may think that you're grouping single statements from different levels, but actually you're not.
   
   I see and understand your point here. I will get back to you with this one.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071355567


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;

Review Comment:
   good catch, thanks. 
   I've added method call to `localGroupCodeLength`.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1085531144


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   @tsreaper 
   Sorry for delay on this one.
   
   Regarding:
   > I mean something like the following.
   
   I understand your suggestion, I was thinking about similar thing at the beginning of designing the solution.
   
   Let me try to rephrase what I think is main difference in yours and mine proposition.
   
   You are proposing to have single Visitor instance that will recursively parse every statement/block.  Every statement extracted from particular "level" will be added to `blocks` map, where key is "context" that identifies the "level/block" that is currently processed.
   
   In my proposition, I'm creating new Visitor for every new nested "block/level". Every processor maintains its own list of extracted statements. At the end I'm traversing through all visitors and gather all extracted blocks -> mergeBlock method (not the best name I agree).
   
   
   What I also had the initial ` if (ctx.WHILE() != null) {` and ` } else if (ctx.IF() != null) {` checks were actually only to set context name to `IF` or `WHILE` i kinda thought it would be beneficial for debugging but I guess it adds the complexity here.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1090568909


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,474 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables, IF and
+ * WHILE statements and extract new method for each group making them smaller.
+ *
+ * <p>BlockStatementGrouper 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>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *     myFun_rewriteGroup4(a, b);
+ *     myFun_rewriteGroup5(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup4 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_1_rewriteGroup3(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup5 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_1_rewriteGroup3 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(maxMethodLength, parameters);
+        CommonTokenStream tokenStream =
+                new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+        JavaParser javaParser = new JavaParser(tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        TokenStreamRewriter rewriter = new TokenStreamRewriter(tokenStream);
+        visitor.visitStatement(javaParser.statement(), context, rewriter);
+
+        visitor.rewrite();
+        Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups = visitor.groups;
+
+        Map<String, List<String>> groupStrings = new HashMap<>(groups.size());
+        for (Entry<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> group :
+                groups.entrySet()) {
+            List<String> collectedStringGroups =
+                    group.getValue().getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+
+            groupStrings.put(group.getKey(), collectedStringGroups);
+        }
+
+        return new RewriteGroupedCode(rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor {
+
+        private final Map<String, Pair<TokenStreamRewriter, List<LocalGroupElement>>> groups =
+                new HashMap<>();
+
+        private final long maxMethodLength;
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(long maxMethodLength, String parameters) {
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        public void visitStatement(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+
+            if (ctx.getChildCount() == 0) {
+                return;
+            }
+
+            // For these statements here we want to process all "branches" separately, for example
+            // TRUE and FALSE branch of IF/ELSE block.
+            // each statement can be rewritten and extracted.
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        String localContext = String.format("%s_%d", context, counter++);
+                        groupBlock(statement, localContext, rewriter);
+                    }
+                }
+            } else {
+                // The block did not start from IF/ELSE/WHILE statement
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx, context, rewriter);
+                }
+            }
+        }
+
+        // Group continuous block of statements together. If Statement is an IF/ELSE/WHILE,
+        // its body can be further grouped by recursive call to visitStatement method.
+        private void groupBlock(
+                StatementContext ctx, String context, TokenStreamRewriter rewriter) {
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + this.counter++;
+
+                    CommonTokenStream tokenStream =
+                            new CommonTokenStream(
+                                    new JavaLexer(
+                                            CharStreams.fromString(
+                                                    CodeSplitUtil.getContextString(statement))));

Review Comment:
   Please remember that that comment was created for previous implementation, where I was using multiple instances of Splitter and Grouper visitors. I refactor both classes so now they are using recursive calls to `visit...` methods. 
   
   Regarding the "repeatedly parse" part.
   Statement is processed by Splitter and Grouper once. I'm not sure how it looks from the Antlr4 but I guess it will be lie so:
   1. Initial parse for Splitter (part of original block statement to rewrite)
   2. Initial parser for Grouper  (part of original block statement to rewrite)
   3. possibly a third time when we create a new rewriter instane for `RewriteContextGroupElement`
   
   I think that my main point was that in my implementation I don't parse entire class again every time when `IfStatementRewriter` is doing another round to process extracted methods. At least this is what I think is happening in `IfStatementRewriter` and that was the reason you had
   
   ```
           while (visitor.hasRewrite()) {
               visitor = new IfStatementVisitor(rewriterCode);
               rewriterCode = visitor.rewriteAndGetCode();
           }
   ```
   
   I think that in my case, the number of times that statement will be parsed by antlr or processed by Splitter/Grouper is constant, whereas in `IfStatementRewriter` it depends on the number of extracted methods.
   
   But I want to be crystal clear here. I dont want to provoke a discussion about the performance especially when you said that with original approach you never had issue regarding the performance - I get that. When I start implementing my solution this was not my goal. Its a side effect of the design approach I took  i think. 
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1088399790


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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;
+    }

Review Comment:
   Fixed, now it returns only String.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1412469928

   Hi @tsreaper 
   thanks for kind words. I'm really happy that we are getting closer to merge this change.
   
   I was using Maciej Bryński's SQL example from FLINK-27246 ticket to verify if my change here solves the original problem.
   I will add it to CodeSplitITCase in following days.


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1370778644

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1371926802

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] flinkbot commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "619b8ca98caded056e851b29e6345e6ec854329a",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "619b8ca98caded056e851b29e6345e6ec854329a",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 619b8ca98caded056e851b29e6345e6ec854329a UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@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.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1371076557

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1405227888

   Hi @tsreaper 
   I'm currently working on the refactoring to include your suggestions, it does look promising. I should have new version by the end of this week.
   
   I have a question though.
   
   Can we assume that variables/fields used in IF and WHILE conditions will be always class member variables? I'm looking at `DeclarationRewriter` and I wonder if we take such assumption or can IF/WHILE condition arguments can come from method parameters?
   
   If they can come from method parameters and if they are primitive values, then we should not rewrite such a method.
   I don't think that original IfStatementRewriter deal with that but maybe I don't see it.
   
   What do you think?
   
   
   
   
   


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "tsreaper (via GitHub)" <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1086577308


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   I understand. If it is difficult to refactor this part of code, at least add detailed comments showing that we're only grouping single statements on the same level, not merging different levels together. In `mergeBlocks` you should also assert that two maps do not have the same key.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1371489783

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on pull request #21393: [Draft][FLINK-27246_master] - Split generated java methods - Work in progress

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on PR #21393:
URL: https://github.com/apache/flink/pull/21393#issuecomment-1371918281

   @flinkbot run azure


-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071594183


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementGrouper.java:
##########
@@ -0,0 +1,432 @@
+/*
+ * 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.Token;
+import org.antlr.v4.runtime.TokenStreamRewriter;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Groups end extract single line statements such as operations on fields/local variables and
+ * extract new method for each group making them smaller.
+ *
+ * <p><i>Before</i>
+ *
+ * <pre><code>
+ * {
+ *     a[0] += b[1];
+ *     b[1] += a[1];
+ *     while (counter > 0) {
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ *         counter--;
+ *     }
+ *
+ *     a[2] += b[2];
+ *     b[3] += a[3];
+ * }
+ *
+ * </code></pre>
+ *
+ * <p><i>After</i>
+ *
+ * <pre><code>
+ * {
+ *      myFun_rewriteGroup1(a, b);
+ *      myFun_rewriteGroup2(a, b);
+ * }
+ * </code></pre>
+ *
+ * <p>Where bodies of extracted "methods" are:
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup1 ->
+ *         a[0] += b[1];
+ *         b[1] += a[1];
+ *         while (counter > 0) {
+ *             myFun_rewriteGroup0_rewriteGroup1(a, b);
+ *             counter--;
+ *         }
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup2 ->
+ *         a[2] += b[2];
+ *         b[3] += a[3];
+ * </code></pre>
+ *
+ * <pre><code>
+ *     myFun_rewriteGroup0_rewriteGroup1 ->
+ *         myFun_whileBody0_0(a, b);
+ *         if (a[0] > 0) {
+ *             myFun_whileBody0_0_ifBody0(a, b);
+ *         } else {
+ *             myFun_whileBody0_0_ifBody1(a, b);
+ *         }
+ * </code></pre>
+ */
+@Internal
+public class BlockStatementGrouper {
+
+    private final String code;
+
+    private final long maxMethodLength;
+
+    private final String parameters;
+
+    /**
+     * Initialize new BlockStatementGrouper.
+     *
+     * @param code code block that should be rewritten for statement grouping.
+     * @param maxMethodLength maximal length of the extracted code block.
+     * @param parameters parameters definition that should be used for extracted methods.
+     */
+    public BlockStatementGrouper(String code, long maxMethodLength, String parameters) {
+        this.code = code;
+        this.maxMethodLength = maxMethodLength;
+        this.parameters = parameters;
+    }
+
+    /**
+     * Rewrite code block used for initialization of this object. The code block is grouped into new
+     * methods.
+     *
+     * @param context prefix used for extracted group names.
+     * @return {@link RewriteGroupedCode} representing rewritten code block and containing extracted
+     *     groups with their names and content.
+     */
+    public RewriteGroupedCode rewrite(String context) {
+
+        BlockStatementGrouperVisitor visitor =
+                new BlockStatementGrouperVisitor(code, context, maxMethodLength, parameters);
+        JavaParser javaParser = new JavaParser(visitor.tokenStream);
+        javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+        visitor.visitStatement(javaParser.statement());
+
+        visitor.rewriteToGroups();
+        List<Pair<String, List<LocalGroupElement>>> groups = visitor.getGroups();
+
+        List<Pair<String, List<String>>> groupStrings = new ArrayList<>(groups.size());
+        for (Pair<String, List<LocalGroupElement>> group : groups) {
+            List<String> collectedStringGroups =
+                    group.getValue().stream()
+                            .map(LocalGroupElement::getBody)
+                            .collect(Collectors.toList());
+            groupStrings.add(Pair.of(group.getKey(), collectedStringGroups));
+        }
+
+        return new RewriteGroupedCode(visitor.rewriter.getText(), groupStrings);
+    }
+
+    private static class BlockStatementGrouperVisitor extends JavaParserBaseVisitor<Void> {
+
+        private final List<BlockStatementGrouperVisitor> children = new ArrayList<>();
+
+        private final String context;
+
+        private final CommonTokenStream tokenStream;
+
+        private final TokenStreamRewriter rewriter;
+
+        private final long maxMethodLength;
+
+        private final List<Pair<String, List<LocalGroupElement>>> groups = new ArrayList<>();
+
+        private final String parameters;
+
+        private int counter = 0;
+
+        private BlockStatementGrouperVisitor(
+                String code, String context, long maxMethodLength, String parameters) {
+            this.tokenStream = new CommonTokenStream(new JavaLexer(CharStreams.fromString(code)));
+            this.rewriter = new TokenStreamRewriter(tokenStream);
+            this.context = context;
+            this.maxMethodLength = maxMethodLength;
+            this.parameters = parameters;
+        }
+
+        @Override
+        public Void visitStatement(StatementContext ctx) {
+
+            if (ctx.getChildCount() == 0) {
+                return null;
+            }
+
+            if (ctx.WHILE() != null || ctx.IF() != null || ctx.ELSE() != null) {
+                for (StatementContext statement : ctx.statement()) {
+                    if (shouldExtract(statement)) {
+                        groupBlock(statement);
+                    }
+                }
+            } else {
+                if (shouldExtract(ctx)) {
+                    groupBlock(ctx);
+                }
+            }
+
+            return null;
+        }
+
+        public List<Pair<String, List<LocalGroupElement>>> getGroups() {
+            List<Pair<String, List<LocalGroupElement>>> groupsTmp = new ArrayList<>();
+            for (BlockStatementGrouperVisitor child : children) {
+                groupsTmp.addAll(child.getGroups());
+            }
+
+            groupsTmp.addAll(this.groups);
+
+            return groupsTmp;
+        }
+
+        private void addGroups(List<Pair<String, List<LocalGroupElement>>> groups) {
+            this.groups.addAll(groups);
+        }
+
+        private void groupBlock(StatementContext ctx) {
+            int localCounter = this.counter++;
+            int localGroupCodeLength = 0;
+            List<LocalGroupElement> localGroup = new ArrayList<>();
+            for (BlockStatementContext bsc : ctx.block().blockStatement()) {
+
+                StatementContext statement = bsc.statement();
+                if (statement.IF() != null
+                        || statement.ELSE() != null
+                        || statement.WHILE() != null) {
+                    String localContext = context + "_rewriteGroup" + localCounter;
+                    BlockStatementGrouperVisitor visitor =
+                            new BlockStatementGrouperVisitor(
+                                    CodeSplitUtil.getContextString(statement),
+                                    localContext,
+                                    maxMethodLength,
+                                    parameters);
+                    JavaParser javaParser = new JavaParser(visitor.tokenStream);
+                    javaParser.getInterpreter().setPredictionMode(PredictionMode.SLL);
+                    visitor.visitStatement(javaParser.statement());
+
+                    localGroup.add(new RewriteContextGroupElement(statement, visitor.rewriter));
+                    children.add(visitor);
+                    localCounter = this.counter++;
+                } else {
+
+                    if (localGroupCodeLength + 1 + bsc.getText().length() <= maxMethodLength) {
+                        localGroup.add(new ContextGroupElement(bsc));
+                        localGroupCodeLength += bsc.getText().length();
+                    } else {
+                        if (localGroup.size() > 1
+                                || (localGroup.size() == 1
+                                        && canGroupAsSingleStatement(
+                                                localGroup.get(0).getContext()))) {
+                            String localContext = context + "_rewriteGroup" + localCounter;
+                            groups.add(Pair.of(localContext, localGroup));
+                            localCounter = this.counter++;
+                            localGroup = new ArrayList<>();
+                        }
+                        localGroupCodeLength = bsc.getText().length();

Review Comment:
   I've applied your suggestion on my local branch and pushed.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] tsreaper commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
tsreaper commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068930909


##########
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:
   Why are you adding blocks to the recursively created `BlockStatementVisitor`? This will cause the problem that statements in `extractedSingleBlocks` might not belong to the same block.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1068750626


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,417 @@
+/*
+ * 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;
+import java.util.stream.Collectors;
+
+/**
+ * Extract statements from IFs, ELSEs and WILEs blocks making them smaller.
+ *
+ * <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 List<BlockStatementSplitter> children = new ArrayList<>();
+
+    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());
+
+        Map<String, List<String>> extractedBlocks = new HashMap<>();
+        for (BlockStatementVisitor child : visitor.children) {
+            int counter = 0;
+            for (ContextTextPair extractedBlock : child.extractedSingleBlocks) {
+                ParserRuleContext parserRuleContext = extractedBlock.parserRuleContext;
+                if (parserRuleContext instanceof BlockStatementContext) {
+                    StatementContext statement =
+                            ((BlockStatementContext) parserRuleContext).statement();
+
+                    if (statement != null
+                            && (statement.IF() != null
+                                    || statement.ELSE() != null
+                                    || statement.WHILE() != null)) {
+
+                        BlockStatementSplitter splitter =
+                                new BlockStatementSplitter(
+                                        extractedBlock.ruleText, this.parameters);
+                        Map<String, List<String>> rewrite =
+                                splitter.extractBlocks(child.context + "_" + counter++);
+                        this.children.add(splitter);
+
+                        mergeBlocks(rewrite, extractedBlocks);
+                    }
+                }
+            }
+        }
+
+        Map<String, List<String>> localBlocks = visitor.getAllBlocks();
+        mergeBlocks(localBlocks, extractedBlocks);
+
+        return extractedBlocks;
+    }
+
+    /**
+     * 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() {
+
+        for (BlockStatementSplitter child : children) {
+            child.visitor.rewrite();
+        }
+        visitor.rewrite();
+
+        Map<String, String> rewriteBlocks = new HashMap<>();
+        for (BlockStatementSplitter child : children) {
+            Map<String, String> childRewriteBlocks = child.rewriteBlock();
+            for (Entry<String, String> entry : childRewriteBlocks.entrySet()) {
+                rewriteBlocks.merge(
+                        entry.getKey(),
+                        entry.getValue(),
+                        (s, s2) -> {
+                            throw new RuntimeException(
+                                    String.format(
+                                            "Override rewritten block  for key %s. Blocks are %s -> %s",
+                                            entry.getKey(), s, s2));
+                        });
+            }
+        }
+
+        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<ContextTextPair> 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 {

Review Comment:
   Hi @tsreaper 
   In fact we don't need `ctx.ELSE()` branch at all. I will try to explain why.
   
   The initial input statement for `BlockStatementSplitter` can be two things:
   
   - a statement that is represented like this:
   
   ```
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   ```
   
   - a block, that will be represented as this:
   
   ```
   {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   }
   ```
   
   Mind that block has additional brackets `{ }` at the beginning and at the end. 
   
   When `BlockStatementSplitter` is called from `BlockStatementRewriter` it will always be called by passing the statement, so without brackets `{ }`. The statement will never start from `else`, it will start from IF or WHILE (this is what we detect for now) or it will be what I call a "single line statement", meaning operation on a field, for example `a[2] = 1;`.
   
   Lets look at the example. The `BlockStatementRewriter` called `BlockStatementSplitter::visitStatement` with below statement:
   
   ```
   if (a[0] == 0) {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   } else {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   
   For this, the condition `if (ctx.WHILE() != null)` will be true. 
   In this branch we will iterate through every `ctx.statement()`
   The first item would be:
   
   ```
   {
   	if (a[2] == 0) {
   		a[2] = 1;
   	} else {
   		a[2] = b[2];
   	}
   }
   ```
   So the `TRUE` branch of the first level IF/ELSE expression. We will create a new context to indicate that we will process an IF(possibly also ELSE) statement and we will create new `BlockStatementVisitor` and call visitStatement method to process that block, since it can contain more nested structures we would like to rewrite. Mind that now the input argument is a "block", it starts and ends with brackets `{ }`
   
   The second item in the iteration routine will be this:
   ```
   {
   	a[0] = b[0];
   	a[1] = b[1];
   }
   ```
   So the `FALSE` branch of the first level IF/ELSE expression. Same as for TRUE, we will create new context and `BlockStatementVisitor` to process that block since it can contain more nested structures we would like to rewrite. In this case the input also would be the "block" since it starts and ends with brackets `{ }`, same as for TRUE branch.
   
   In both new Visitors, conditions `ctx.WHILE() != null` and `ctx.IF() != null` will be false since now we have a "block", it starts from `{`. So we will land at `if (ctx.block() != null) ` line ~224 of `BlockStatementSplitter.java`. Since this is a block now, we need to iterate over all its statements hence `for (BlockStatementContext bsc : ctx.block().blockStatement())`
   
   For `TRUE` branch from the initial statement it will be only one item:
   ```
   if (a[2] == 0) {
   	a[2] = 1;
   } else {
   	a[2] = b[2];
   }
   ```
   We will create new Visitor and process this statement further.
   
   For `FALSE` branch, from the initial statement it will contain two elements. Those will be "single line elements":
   - a[0] = b[0];
   - a[1] = b[1];
   
   Those will be added to `extractedSingleBlocks` blocks and extracted to separate method.
   
   I hope that clear the logic a little bit.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by GitBox <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1071334951


##########
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(

Review Comment:
   maybe `mergeBlocks` is not the best name in this case.



-- 
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-unsubscribe@flink.apache.org

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


[GitHub] [flink] kristoffSC commented on a diff in pull request #21393: [FLINK-27246][TableSQL/Runtime][master] - Include WHILE blocks in split generated java methods

Posted by "kristoffSC (via GitHub)" <gi...@apache.org>.
kristoffSC commented on code in PR #21393:
URL: https://github.com/apache/flink/pull/21393#discussion_r1085531144


##########
flink-table/flink-table-code-splitter/src/main/java/org/apache/flink/table/codesplit/BlockStatementSplitter.java:
##########
@@ -0,0 +1,326 @@
+/*
+ * 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 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) {

Review Comment:
   @tsreaper 
   Sorry for delay on this one.
   
   Regarding:
   > I mean something like the following.
   
   I understand your suggestion, I was thinking about similar thing at the beginning of designing the solution.
   
   Let me try to rephrase what I think is main difference in yours and mine proposition.
   
   You are proposing to have single Visitor instance that will recursively parse every statement/block.  Every statement extracted from particular "level" will be added to `blocks` map, where key is "context" that identifies the "level/block" that is currently processed.
   
   In my proposition, I'm creating new Visitor for every new nested "block/level". Every processor maintains its own list of extracted statements. At the end I'm traversing through all visitors and gather all extracted blocks -> mergeBlock method (not the best name I agree).
   
   
   Also why I had the initial ` if (ctx.WHILE() != null) {` and ` } else if (ctx.IF() != null) {` checks were actually was only to set context name to `IF` or `WHILE` i kinda thought it would be beneficial for debugging but I guess it adds the complexity here.
   
   
   



-- 
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-unsubscribe@flink.apache.org

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