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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to