You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/05/16 02:53:21 UTC

[6/7] impala git commit: IMPALA-7025: ignore resources in some planner test

IMPALA-7025: ignore resources in some planner test

The issue was that the tablesample test verified the mem-estimate
number, which depends on file sizes, which can vary slightly between
data loads.

Instead of trying to tweak the test to avoid the issue, instead provide
a mechanism to ignore the exact values of resources in planner tests
where they are not significant.

Testing:
Manually modified some values in tablesample.test, made sure that the
test still passed. Manually modified the partition count in the
expected output, made sure that the test failed.

Change-Id: I91e3e416ec6242fbf22d9f566fdd1ce225cb16ac
Reviewed-on: http://gerrit.cloudera.org:8080/10410
Reviewed-by: Tim Armstrong <ta...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


Project: http://git-wip-us.apache.org/repos/asf/impala/repo
Commit: http://git-wip-us.apache.org/repos/asf/impala/commit/be2d61a7
Tree: http://git-wip-us.apache.org/repos/asf/impala/tree/be2d61a7
Diff: http://git-wip-us.apache.org/repos/asf/impala/diff/be2d61a7

Branch: refs/heads/master
Commit: be2d61a7e2e21f5837c946641b094651c4461aa4
Parents: caf275c
Author: Tim Armstrong <ta...@cloudera.com>
Authored: Mon May 14 17:40:45 2018 -0700
Committer: Impala Public Jenkins <im...@cloudera.com>
Committed: Wed May 16 02:23:52 2018 +0000

----------------------------------------------------------------------
 .../org/apache/impala/planner/PlannerTest.java  |  49 +++++----
 .../apache/impala/planner/PlannerTestBase.java  |  69 +++++++++----
 .../org/apache/impala/testutil/TestUtils.java   | 102 +++++++++++--------
 3 files changed, 136 insertions(+), 84 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/be2d61a7/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
index 8b9166f..2667e74 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTest.java
@@ -34,6 +34,7 @@ import org.junit.Assume;
 import org.junit.Test;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.Lists;
 
 // All planner tests, except for S3 specific tests should go here.
@@ -54,9 +55,8 @@ public class PlannerTest extends PlannerTestBase {
     // Tests that constant folding is applied to all relevant PlanNodes and DataSinks.
     // Note that not all Exprs are printed in the explain plan, so validating those
     // via this test is currently not possible.
-    TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("constant-folding", options);
+    runPlannerTestFile("constant-folding",
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test
@@ -149,10 +149,9 @@ public class PlannerTest extends PlannerTestBase {
 
   @Test
   public void testFkPkJoinDetection() {
-    TQueryOptions options = defaultQueryOptions();
     // The FK/PK detection result is included in EXTENDED or higher.
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("fk-pk-join-detection", options);
+    runPlannerTestFile("fk-pk-join-detection",
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test
@@ -269,7 +268,8 @@ public class PlannerTest extends PlannerTestBase {
   public void testDisableCodegenOptimization() {
     TQueryOptions options = new TQueryOptions();
     options.setDisable_codegen_rows_threshold(3000);
-    runPlannerTestFile("disable-codegen", options, false);
+    runPlannerTestFile("disable-codegen", options,
+        ImmutableSet.of(PlannerTestOption.INCLUDE_EXPLAIN_HEADER));
   }
 
   @Test
@@ -316,9 +316,8 @@ public class PlannerTest extends PlannerTestBase {
 
   @Test
   public void testParquetFiltering() {
-    TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("parquet-filtering", options);
+    runPlannerTestFile("parquet-filtering",
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test
@@ -426,9 +425,11 @@ public class PlannerTest extends PlannerTestBase {
   public void testResourceRequirements() {
     // Tests the resource requirement computation from the planner.
     TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
     options.setNum_scanner_threads(1); // Required so that output doesn't vary by machine
-    runPlannerTestFile("resource-requirements", options, false);
+    runPlannerTestFile("resource-requirements", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN,
+            PlannerTestOption.INCLUDE_EXPLAIN_HEADER,
+            PlannerTestOption.VALIDATE_RESOURCES));
   }
 
   @Test
@@ -438,7 +439,10 @@ public class PlannerTest extends PlannerTestBase {
     TQueryOptions options = defaultQueryOptions();
     options.setExplain_level(TExplainLevel.EXTENDED);
     options.setNum_scanner_threads(1); // Required so that output doesn't vary by machine
-    runPlannerTestFile("spillable-buffer-sizing", options, false);
+    runPlannerTestFile("spillable-buffer-sizing", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN,
+            PlannerTestOption.INCLUDE_EXPLAIN_HEADER,
+            PlannerTestOption.VALIDATE_RESOURCES));
   }
 
   @Test
@@ -449,22 +453,24 @@ public class PlannerTest extends PlannerTestBase {
     options.setExplain_level(TExplainLevel.EXTENDED);
     options.setNum_scanner_threads(1); // Required so that output doesn't vary by machine
     options.setMax_row_size(8L * 1024L * 1024L);
-    runPlannerTestFile("max-row-size", options, false);
+    runPlannerTestFile("max-row-size", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN,
+          PlannerTestOption.INCLUDE_EXPLAIN_HEADER,
+          PlannerTestOption.VALIDATE_RESOURCES));
   }
 
   @Test
   public void testSortExprMaterialization() {
     addTestFunction("TestFn", Lists.newArrayList(Type.DOUBLE), false);
-    TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("sort-expr-materialization", options);
+    runPlannerTestFile("sort-expr-materialization",
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test
   public void testTableSample() {
     TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("tablesample", options);
+    runPlannerTestFile("tablesample", options,
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test
@@ -479,9 +485,8 @@ public class PlannerTest extends PlannerTestBase {
 
   @Test
   public void testPartitionPruning() {
-    TQueryOptions options = defaultQueryOptions();
-    options.setExplain_level(TExplainLevel.EXTENDED);
-    runPlannerTestFile("partition-pruning", options);
+    runPlannerTestFile("partition-pruning",
+        ImmutableSet.of(PlannerTestOption.EXTENDED_EXPLAIN));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/impala/blob/be2d61a7/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
index 3efbc63..bcd1fc3 100644
--- a/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
+++ b/fe/src/test/java/org/apache/impala/planner/PlannerTestBase.java
@@ -45,6 +45,7 @@ import org.apache.impala.testutil.TestFileParser;
 import org.apache.impala.testutil.TestFileParser.Section;
 import org.apache.impala.testutil.TestFileParser.TestCase;
 import org.apache.impala.testutil.TestUtils;
+import org.apache.impala.testutil.TestUtils.ResultFilter;
 import org.apache.impala.thrift.ImpalaInternalServiceConstants;
 import org.apache.impala.thrift.TDescriptorTable;
 import org.apache.impala.thrift.TExecRequest;
@@ -396,8 +397,8 @@ public class PlannerTestBase extends FrontendTestBase {
    * of 'testCase'.
    */
   private void runTestCase(TestCase testCase, StringBuilder errorLog,
-      StringBuilder actualOutput, String dbName, boolean ignoreExplainHeader)
-      throws CatalogException {
+      StringBuilder actualOutput, String dbName,
+      Set<PlannerTestOption> testOptions) throws CatalogException {
     String query = testCase.getQuery();
     LOG.info("running query " + query);
     if (query.isEmpty()) {
@@ -411,16 +412,16 @@ public class PlannerTestBase extends FrontendTestBase {
     queryCtx.client_request.query_options = testCase.getOptions();
     // Test single node plan, scan range locations, and column lineage.
     TExecRequest singleNodeExecRequest = testPlan(testCase, Section.PLAN, queryCtx.deepCopy(),
-        ignoreExplainHeader, errorLog, actualOutput);
+        testOptions, errorLog, actualOutput);
     validateTableIds(singleNodeExecRequest);
     checkScanRangeLocations(testCase, singleNodeExecRequest, errorLog, actualOutput);
     checkColumnLineage(testCase, singleNodeExecRequest, errorLog, actualOutput);
     checkLimitCardinality(query, singleNodeExecRequest, errorLog);
     // Test distributed plan.
-    testPlan(testCase, Section.DISTRIBUTEDPLAN, queryCtx.deepCopy(), ignoreExplainHeader,
+    testPlan(testCase, Section.DISTRIBUTEDPLAN, queryCtx.deepCopy(), testOptions,
         errorLog, actualOutput);
     // test parallel plans
-    testPlan(testCase, Section.PARALLELPLANS, queryCtx.deepCopy(), ignoreExplainHeader,
+    testPlan(testCase, Section.PARALLELPLANS, queryCtx.deepCopy(), testOptions,
         errorLog, actualOutput);
   }
 
@@ -476,11 +477,10 @@ public class PlannerTestBase extends FrontendTestBase {
    * Returns the produced exec request or null if there was an error generating
    * the plan.
    *
-   * If ignoreExplainHeader is true, the explain header with warnings and resource
-   * estimates is stripped out.
+   * testOptions control exactly how the plan is generated and compared.
    */
   private TExecRequest testPlan(TestCase testCase, Section section,
-      TQueryCtx queryCtx, boolean ignoreExplainHeader,
+      TQueryCtx queryCtx, Set<PlannerTestOption> testOptions,
       StringBuilder errorLog, StringBuilder actualOutput) {
     String query = testCase.getQuery();
     queryCtx.client_request.setStmt(query);
@@ -516,7 +516,9 @@ public class PlannerTestBase extends FrontendTestBase {
     if (execRequest == null) return null;
 
     String explainStr = explainBuilder.toString();
-    if (ignoreExplainHeader) explainStr = removeExplainHeader(explainStr);
+    if (!testOptions.contains(PlannerTestOption.INCLUDE_EXPLAIN_HEADER)) {
+      explainStr = removeExplainHeader(explainStr);
+    }
     actualOutput.append(explainStr);
     LOG.info(section.toString() + ":" + explainStr);
     if (expectedErrorMsg != null) {
@@ -524,8 +526,13 @@ public class PlannerTestBase extends FrontendTestBase {
           "\nExpected failure, but query produced %s.\nQuery:\n%s\n\n%s:\n%s",
           section, query, section, explainStr));
     } else {
+      List<ResultFilter> resultFilters =
+          Lists.<ResultFilter>newArrayList(TestUtils.FILE_SIZE_FILTER);
+      if (!testOptions.contains(PlannerTestOption.VALIDATE_RESOURCES)) {
+        resultFilters.addAll(TestUtils.RESOURCE_FILTERS);
+      }
       String planDiff = TestUtils.compareOutput(
-          Lists.newArrayList(explainStr.split("\n")), expectedPlan, true, true);
+          Lists.newArrayList(explainStr.split("\n")), expectedPlan, true, resultFilters);
       if (!planDiff.isEmpty()) {
         errorLog.append(String.format(
             "\nSection %s of query:\n%s\n\n%s", section, query, planDiff));
@@ -587,7 +594,8 @@ public class PlannerTestBase extends FrontendTestBase {
     if (expectedLocations.size() > 0 && locationsStr != null) {
       // Locations' order does not matter.
       String result = TestUtils.compareOutput(
-          Lists.newArrayList(locationsStr.split("\n")), expectedLocations, false, false);
+          Lists.newArrayList(locationsStr.split("\n")), expectedLocations, false,
+          Collections.<TestUtils.ResultFilter>emptyList());
       if (!result.isEmpty()) {
         errorLog.append("section " + Section.SCANRANGELOCATIONS + " of query:\n"
             + query + "\n" + result);
@@ -749,23 +757,46 @@ public class PlannerTestBase extends FrontendTestBase {
     return explain;
   }
 
+  /**
+   * Assorted binary options that alter the behaviour of planner tests, generally
+   * enabling additional more-detailed checks.
+   */
+  protected static enum PlannerTestOption {
+    // Generate extended explain plans (default is STANDARD).
+    EXTENDED_EXPLAIN,
+    // Include the header of the explain plan (default is to strip the explain header).
+    INCLUDE_EXPLAIN_HEADER,
+    // Validate the values of resource requirements (default is to ignore differences
+    // in resource values).
+    VALIDATE_RESOURCES,
+  }
+
   protected void runPlannerTestFile(String testFile, TQueryOptions options) {
-    runPlannerTestFile(testFile, options, true);
+    runPlannerTestFile(testFile, "default", options,
+        Collections.<PlannerTestOption>emptySet());
   }
 
   protected void runPlannerTestFile(String testFile, TQueryOptions options,
-      boolean ignoreExplainHeader) {
-    runPlannerTestFile(testFile, "default", options, ignoreExplainHeader);
+      Set<PlannerTestOption> testOptions) {
+    runPlannerTestFile(testFile, "default", options, testOptions);
+  }
+
+  protected void runPlannerTestFile(
+      String testFile, Set<PlannerTestOption> testOptions) {
+    runPlannerTestFile(testFile, "default", defaultQueryOptions(), testOptions);
   }
 
   private void runPlannerTestFile(String testFile, String dbName, TQueryOptions options,
-      boolean ignoreExplainHeader) {
+        Set<PlannerTestOption> testOptions) {
     String fileName = testDir_.resolve(testFile + ".test").toString();
     if (options == null) {
       options = defaultQueryOptions();
     } else {
       options = mergeQueryOptions(defaultQueryOptions(), options);
     }
+    if (testOptions.contains(PlannerTestOption.EXTENDED_EXPLAIN)) {
+      options.setExplain_level(TExplainLevel.EXTENDED);
+    }
     TestFileParser queryFileParser = new TestFileParser(fileName, options);
     StringBuilder actualOutput = new StringBuilder();
 
@@ -782,7 +813,7 @@ public class PlannerTestBase extends FrontendTestBase {
         actualOutput.append("\n");
       }
       try {
-        runTestCase(testCase, errorLog, actualOutput, dbName, ignoreExplainHeader);
+        runTestCase(testCase, errorLog, actualOutput, dbName, testOptions);
       } catch (CatalogException e) {
         errorLog.append(String.format("Failed to plan query\n%s\n%s",
             testCase.getQuery(), e.getMessage()));
@@ -808,10 +839,12 @@ public class PlannerTestBase extends FrontendTestBase {
   }
 
   protected void runPlannerTestFile(String testFile) {
-    runPlannerTestFile(testFile, "default", null, true);
+    runPlannerTestFile(testFile, "default", defaultQueryOptions(),
+        Collections.<PlannerTestOption>emptySet());
   }
 
   protected void runPlannerTestFile(String testFile, String dbName) {
-    runPlannerTestFile(testFile, dbName, null, true);
+    runPlannerTestFile(testFile, dbName, defaultQueryOptions(),
+        Collections.<PlannerTestOption>emptySet());
   }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/be2d61a7/fe/src/test/java/org/apache/impala/testutil/TestUtils.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/testutil/TestUtils.java b/fe/src/test/java/org/apache/impala/testutil/TestUtils.java
index 629c44b..cbc3d1b 100644
--- a/fe/src/test/java/org/apache/impala/testutil/TestUtils.java
+++ b/fe/src/test/java/org/apache/impala/testutil/TestUtils.java
@@ -20,9 +20,11 @@ import java.io.StringReader;
 import java.io.StringWriter;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Calendar;
 import java.util.Collections;
 import java.util.Date;
+import java.util.List;
 import java.util.Map;
 import java.util.Scanner;
 import java.util.TimeZone;
@@ -56,7 +58,12 @@ public class TestUtils {
   // than a literal
   private final static String REGEX_AGAINST_ACTUAL = "regex:";
 
-  interface ResultFilter {
+  // Regexes that match various elements in plan.
+  private final static String NUMBER_REGEX = "\\d+(\\.\\d+)?";
+  private final static String BYTE_SUFFIX_REGEX = "[KMGT]?B";
+  private final static String BYTE_VALUE_REGEX = NUMBER_REGEX + BYTE_SUFFIX_REGEX;
+
+  public interface ResultFilter {
     public boolean matches(String input);
     public String transform(String input);
   }
@@ -82,41 +89,54 @@ public class TestUtils {
     }
   }
 
-  static PathFilter[] pathFilterList_ = {
-    new PathFilter("hdfs:"),
-    new PathFilter("file: ")
-  };
 
-  // File size could vary from run to run. For example, the parquet file header size
-  // or column metadata size could change if the Impala version changes. That doesn't
-  // mean anything is wrong with the plan, so we want to filter the file size out.
-  static class FileSizeFilter implements ResultFilter {
-    private final static String BYTE_FILTER = "[KMGT]?B";
-    private final static String NUMBER_FILTER = "\\d+(\\.\\d+)?";
-    private final static String FILTER_KEY = " size=";
+  /**
+   * Filter to ignore the value from elements in the format key=value.
+   */
+  public static class IgnoreValueFilter implements ResultFilter {
+    // Literal string containing the key name.
+    private final String keyPrefix;
+    private final String valueRegex;
+
+    /**
+     * Create a filter that ignores the value from key value pairs where the key is
+     * the literal 'key' value and the value matches 'valueRegex'.
+     */
+    public IgnoreValueFilter(String key, String valueRegex) {
+      // Include leading space to avoid matching partial keys, e.g. if key is "bar" we
+      // don't want to match "foobar=".
+      this.keyPrefix = " " + key + "=";
+      this.valueRegex = valueRegex;
+    }
 
-    public boolean matches(String input) { return input.contains(FILTER_KEY); }
+    public boolean matches(String input) { return input.contains(keyPrefix); }
 
     public String transform(String input) {
-      return input.replaceAll(FILTER_KEY + NUMBER_FILTER + BYTE_FILTER, FILTER_KEY);
+      return input.replaceAll(keyPrefix + valueRegex, keyPrefix);
     }
   }
-
-  static FileSizeFilter fileSizeFilter_ = new FileSizeFilter();
+  // File size could vary from run to run. For example, the parquet file header size
+  // or column metadata size could change if the Impala version changes. That doesn't
+  // mean anything is wrong with the plan, so we want to filter the file size out.
+  public static final IgnoreValueFilter FILE_SIZE_FILTER =
+      new IgnoreValueFilter("size", BYTE_VALUE_REGEX);
 
   // Ignore the exact estimated row count, which depends on the file sizes.
-  static class ScanRangeRowCountFilter implements ResultFilter {
-    private final static String NUMBER_FILTER = "\\d+(\\.\\d+)?";
-    private final static String FILTER_KEY = " max-scan-range-rows=";
-
-    public boolean matches(String input) { return input.contains(FILTER_KEY); }
+  static IgnoreValueFilter SCAN_RANGE_ROW_COUNT_FILTER =
+      new IgnoreValueFilter("max-scan-range-rows", NUMBER_REGEX);
 
-    public String transform(String input) {
-      return input.replaceAll(FILTER_KEY + NUMBER_FILTER, FILTER_KEY);
-    }
-  }
+  // Filters that are always applied
+  private static final List<ResultFilter> DEFAULT_FILTERS = Arrays.<ResultFilter>asList(
+    SCAN_RANGE_ROW_COUNT_FILTER, new PathFilter("hdfs:"), new PathFilter("file: "));
 
-  static ScanRangeRowCountFilter scanRangeRowCountFilter_ = new ScanRangeRowCountFilter();
+  // Filters that ignore the values of resource requirements that appear in
+  // "EXTENDED" and above explain plans.
+  public static final List<ResultFilter> RESOURCE_FILTERS = Arrays.<ResultFilter>asList(
+      new IgnoreValueFilter("mem-estimate", BYTE_VALUE_REGEX),
+      new IgnoreValueFilter("mem-reservation", BYTE_VALUE_REGEX),
+      new IgnoreValueFilter("thread-reservation", NUMBER_REGEX),
+      new IgnoreValueFilter("Memory", BYTE_VALUE_REGEX),
+      new IgnoreValueFilter("Threads", NUMBER_REGEX));
 
   /**
    * Do a line-by-line comparison of actual and expected output.
@@ -126,12 +146,14 @@ public class TestUtils {
    * the expected line (ignoring the expectedFilePrefix prefix).
    * If orderMatters is false, we consider actual to match expected if they
    * both contains the same output lines regardless of order.
+   * lineFilters is a list of filters that are applied to corresponding lines in the
+   * actual and expected output if the filter matches the expected output.
    *
    * @return an error message if actual does not match expected, "" otherwise.
    */
   public static String compareOutput(
       ArrayList<String> actual, ArrayList<String> expected, boolean orderMatters,
-      boolean filterFileSize) {
+      List<ResultFilter> lineFilters) {
     if (!orderMatters) {
       Collections.sort(actual);
       Collections.sort(expected);
@@ -141,26 +163,18 @@ public class TestUtils {
     for (int i = 0; i < maxLen; ++i) {
       String expectedStr = expected.get(i).trim();
       String actualStr = actual.get(i);
-      // Filter out contents that change run to run but don't affect compare result.
+      // Apply all default and caller-supplied filters to the expected and actual output.
       boolean containsPrefix = false;
-      for (PathFilter filter: pathFilterList_) {
-        if (filter.matches(expectedStr)) {
-          containsPrefix = true;
-          expectedStr = filter.transform(expectedStr);
-          actualStr = filter.transform(actualStr);
-          break;
+      for (List<ResultFilter> filters:
+          Arrays.<List<ResultFilter>>asList(DEFAULT_FILTERS, lineFilters)) {
+        for (ResultFilter filter: filters) {
+          if (filter.matches(expectedStr)) {
+            containsPrefix = true;
+            expectedStr = filter.transform(expectedStr);
+            actualStr = filter.transform(actualStr);
+          }
         }
       }
-      if (filterFileSize && fileSizeFilter_.matches(expectedStr)) {
-        containsPrefix = true;
-        expectedStr = fileSizeFilter_.transform(expectedStr);
-        actualStr = fileSizeFilter_.transform(actualStr);
-      }
-      if (scanRangeRowCountFilter_.matches(expectedStr)) {
-        containsPrefix = true;
-        expectedStr = scanRangeRowCountFilter_.transform(expectedStr);
-        actualStr = scanRangeRowCountFilter_.transform(actualStr);
-      }
 
       boolean ignoreAfter = false;
       for (int j = 0; j < ignoreContentAfter_.length; ++j) {