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

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

Reply via email to