You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@orc.apache.org by do...@apache.org on 2021/09/03 15:41:20 UTC

[orc] branch main updated: ORC-980: Filter processing respects the case-sensitivity flag (#893)

This is an automated email from the ASF dual-hosted git repository.

dongjoon pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/orc.git


The following commit(s) were added to refs/heads/main by this push:
     new 59c078d  ORC-980: Filter processing respects the case-sensitivity flag (#893)
59c078d is described below

commit 59c078d89d0fe2d2921b37113731cc032efc472e
Author: Pavan Lanka <pl...@apple.com>
AuthorDate: Fri Sep 3 08:40:20 2021 -0700

    ORC-980: Filter processing respects the case-sensitivity flag (#893)
    
    ### What changes were proposed in this pull request?
    
    The filter processing incorrectly ignores the case-sensitivity flag for schema supplied by the reader. This fixes this bug and uses the flag in determining field and vector information.
    
    ### Why are the changes needed?
    
    In the absence of this the read incorrectly fails by performing a case-sensitive match even when case-insensitive match was requested by the reader.
    
    ### How was this patch tested?
    
    Unit Tests were added to verify the failure and subsequently the fix.
---
 .../apache/orc/bench/core/filter/FilterBench.java  |  5 ++-
 .../apache/orc/impl/filter/RowFilterFactory.java   |  1 +
 .../apache/orc/bench/core/filter/TestFilter.java   |  3 +-
 .../org/apache/orc/impl/filter/ATestFilter.java    |  2 +-
 .../org/apache/orc/impl/filter/TestRowFilter.java  |  2 +-
 .../org/apache/orc/impl/OrcFilterContextImpl.java  |  7 +++-
 .../java/org/apache/orc/impl/RecordReaderImpl.java |  1 +
 .../java/org/apache/orc/impl/SchemaEvolution.java  |  4 ++
 .../org/apache/orc/impl/filter/FilterFactory.java  | 26 ++++++++-----
 .../orc/impl/filter/leaf/LeafFilterFactory.java    |  3 +-
 .../orc/impl/reader/tree/StructBatchReader.java    |  4 +-
 .../test/org/apache/orc/TestOrcFilterContext.java  | 11 +++++-
 .../org/apache/orc/TestRowFilteringIOSkip.java     | 43 ++++++++++++++++++++++
 .../apache/orc/impl/TestOrcFilterContextImpl.java  | 10 ++---
 .../org/apache/orc/impl/filter/ATestFilter.java    |  2 +-
 .../org/apache/orc/impl/filter/TestAndFilter.java  |  1 +
 .../org/apache/orc/impl/filter/TestConvFilter.java |  2 +-
 .../org/apache/orc/impl/filter/TestOrFilter.java   |  1 +
 .../apache/orc/impl/filter/leaf/TestFilters.java   |  2 +-
 19 files changed, 102 insertions(+), 28 deletions(-)

diff --git a/java/bench/core/src/java/org/apache/orc/bench/core/filter/FilterBench.java b/java/bench/core/src/java/org/apache/orc/bench/core/filter/FilterBench.java
index d493b67..f7fcb16 100644
--- a/java/bench/core/src/java/org/apache/orc/bench/core/filter/FilterBench.java
+++ b/java/bench/core/src/java/org/apache/orc/bench/core/filter/FilterBench.java
@@ -168,6 +168,7 @@ public class FilterBench implements OrcBenchmark {
           .useSelected(true);
         return FilterFactory.createBatchFilter(options,
                                                FilterBenchUtil.schema,
+                                               false,
                                                OrcFile.Version.CURRENT,
                                                normalize);
       default:
@@ -198,7 +199,7 @@ public class FilterBench implements OrcBenchmark {
       Random rnd = new Random(1024);
       VectorizedRowBatch b = FilterBenchUtil.createBatch(rnd);
       Configuration conf = new Configuration();
-      fc = new OrcFilterContextImpl(FilterBenchUtil.schema).setBatch(b);
+      fc = new OrcFilterContextImpl(FilterBenchUtil.schema, false).setBatch(b);
       Map.Entry<SearchArgument, int[]> r = FilterBenchUtil.createSArg(rnd, b, fInSize);
       SearchArgument sArg = r.getKey();
       expSel = r.getValue();
@@ -247,7 +248,7 @@ public class FilterBench implements OrcBenchmark {
     public void setup() throws FilterFactory.UnSupportedSArgException {
       VectorizedRowBatch b = FilterBenchUtil.createBatch(new Random(1024));
 
-      fc = new OrcFilterContextImpl(FilterBenchUtil.schema).setBatch(b);
+      fc = new OrcFilterContextImpl(FilterBenchUtil.schema, false).setBatch(b);
       Map.Entry<SearchArgument, int[]> r = FilterBenchUtil.createComplexSArg(new Random(1024),
                                                                              b,
                                                                              inSize,
diff --git a/java/bench/core/src/java/org/apache/orc/impl/filter/RowFilterFactory.java b/java/bench/core/src/java/org/apache/orc/impl/filter/RowFilterFactory.java
index 95a75c6..780ca0e 100644
--- a/java/bench/core/src/java/org/apache/orc/impl/filter/RowFilterFactory.java
+++ b/java/bench/core/src/java/org/apache/orc/impl/filter/RowFilterFactory.java
@@ -90,6 +90,7 @@ public class RowFilterFactory {
     LeafFilter f = (LeafFilter) LeafFilterFactory.createLeafVectorFilter(leaf,
                                                                          colIds,
                                                                          readSchema,
+                                                                         false,
                                                                          version,
                                                                          negated);
     return new RowFilter.LeafFilter(f);
diff --git a/java/bench/core/src/test/org/apache/orc/bench/core/filter/TestFilter.java b/java/bench/core/src/test/org/apache/orc/bench/core/filter/TestFilter.java
index 6b29c7d..09f814b 100644
--- a/java/bench/core/src/test/org/apache/orc/bench/core/filter/TestFilter.java
+++ b/java/bench/core/src/test/org/apache/orc/bench/core/filter/TestFilter.java
@@ -48,7 +48,7 @@ public class TestFilter {
   protected final Random rnd = new Random(seed);
   protected final VectorizedRowBatch b = FilterBenchUtil.createBatch(rnd);
   protected final OrcFilterContextImpl fc = (OrcFilterContextImpl)
-    new OrcFilterContextImpl(FilterBenchUtil.schema).setBatch(b);
+    new OrcFilterContextImpl(FilterBenchUtil.schema, false).setBatch(b);
 
   public static Stream<Arguments> filters() {
     return Stream.of(
@@ -108,6 +108,7 @@ public class TestFilter {
             .allowSARGToFilter(true);
           filter = FilterFactory.createBatchFilter(options,
                                                    FilterBenchUtil.schema,
+                                                   false,
                                                    OrcFile.Version.CURRENT,
                                                    normalize);
           break;
diff --git a/java/bench/core/src/test/org/apache/orc/impl/filter/ATestFilter.java b/java/bench/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
index fd55516..06e59bf 100644
--- a/java/bench/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
+++ b/java/bench/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
@@ -39,7 +39,7 @@ public class ATestFilter {
     .addField("f2", TypeDescription.createString())
     .addField("f3p", TypeDescription.createDate())
     .addField("f3h", TypeDescription.createDate());
-  protected final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+  protected final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
 
   protected final VectorizedRowBatch batch = schema.createRowBatch();
 
diff --git a/java/bench/core/src/test/org/apache/orc/impl/filter/TestRowFilter.java b/java/bench/core/src/test/org/apache/orc/impl/filter/TestRowFilter.java
index 08b58bb..a03b32a 100644
--- a/java/bench/core/src/test/org/apache/orc/impl/filter/TestRowFilter.java
+++ b/java/bench/core/src/test/org/apache/orc/impl/filter/TestRowFilter.java
@@ -40,7 +40,7 @@ public class TestRowFilter extends ATestFilter {
   private final TypeDescription schema = TypeDescription.createStruct()
     .addField("f1", TypeDescription.createLong())
     .addField("f2", TypeDescription.createString());
-  final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+  final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
 
   private final VectorizedRowBatch batch = schema.createRowBatch();
 
diff --git a/java/core/src/java/org/apache/orc/impl/OrcFilterContextImpl.java b/java/core/src/java/org/apache/orc/impl/OrcFilterContextImpl.java
index 3f8992c..33eec24 100644
--- a/java/core/src/java/org/apache/orc/impl/OrcFilterContextImpl.java
+++ b/java/core/src/java/org/apache/orc/impl/OrcFilterContextImpl.java
@@ -39,9 +39,11 @@ public class OrcFilterContextImpl implements OrcFilterContext {
   // Cache of field to ColumnVector, this is reset everytime the batch reference changes
   private final Map<String, ColumnVector[]> vectors;
   private final TypeDescription readSchema;
+  private final boolean isSchemaCaseAware;
 
-  public OrcFilterContextImpl(TypeDescription readSchema) {
+  public OrcFilterContextImpl(TypeDescription readSchema, boolean isSchemaCaseAware) {
     this.readSchema = readSchema;
+    this.isSchemaCaseAware = isSchemaCaseAware;
     this.vectors = new HashMap<>();
   }
 
@@ -120,6 +122,7 @@ public class OrcFilterContextImpl implements OrcFilterContext {
   public ColumnVector[] findColumnVector(String name) {
     return vectors.computeIfAbsent(name,
         key -> ParserUtils.findColumnVectors(readSchema,
-            new ParserUtils.StringPosition(key), true, batch));
+                                             new ParserUtils.StringPosition(key),
+                                             isSchemaCaseAware, batch));
   }
 }
diff --git a/java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java b/java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
index 678fd21..89343b9 100644
--- a/java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
+++ b/java/core/src/java/org/apache/orc/impl/RecordReaderImpl.java
@@ -283,6 +283,7 @@ public class RecordReaderImpl implements RecordReader {
     Consumer<OrcFilterContext> filterCallBack = null;
     BatchFilter filter = FilterFactory.createBatchFilter(options,
                                                          evolution.getReaderBaseSchema(),
+                                                         evolution.isSchemaEvolutionCaseAware(),
                                                          fileReader.getFileVersion(),
                                                          false);
     if (filter != null) {
diff --git a/java/core/src/java/org/apache/orc/impl/SchemaEvolution.java b/java/core/src/java/org/apache/orc/impl/SchemaEvolution.java
index eb5beac..168f391 100644
--- a/java/core/src/java/org/apache/orc/impl/SchemaEvolution.java
+++ b/java/core/src/java/org/apache/orc/impl/SchemaEvolution.java
@@ -157,6 +157,10 @@ public class SchemaEvolution {
     return false;
   }
 
+  public boolean isSchemaEvolutionCaseAware() {
+    return isSchemaEvolutionCaseAware;
+  }
+
   public TypeDescription getReaderSchema() {
     return readerSchema;
   }
diff --git a/java/core/src/java/org/apache/orc/impl/filter/FilterFactory.java b/java/core/src/java/org/apache/orc/impl/filter/FilterFactory.java
index 7fbf991..1902f6c 100644
--- a/java/core/src/java/org/apache/orc/impl/filter/FilterFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/filter/FilterFactory.java
@@ -41,14 +41,16 @@ public class FilterFactory {
    * Create a BatchFilter. This considers both the input filter and the SearchArgument filter. If
    * both are available then they are compounded by AND.
    *
-   * @param opts       for reading the file
-   * @param readSchema that should be used
-   * @param version    provides the ORC file version
-   * @param normalize  identifies if the SArg should be normalized or not
+   * @param opts              for reading the file
+   * @param readSchema        that should be used
+   * @param isSchemaCaseAware identifies if the schema is case-sensitive
+   * @param version           provides the ORC file version
+   * @param normalize         identifies if the SArg should be normalized or not
    * @return BatchFilter that represents the SearchArgument or null
    */
   public static BatchFilter createBatchFilter(Reader.Options opts,
                                               TypeDescription readSchema,
+                                              boolean isSchemaCaseAware,
                                               OrcFile.Version version,
                                               boolean normalize) {
     List<BatchFilter> filters = new ArrayList<>(2);
@@ -64,6 +66,7 @@ public class FilterFactory {
                                                                colNames,
                                                                sArg.getLeaves(),
                                                                readSchema,
+                                                               isSchemaCaseAware,
                                                                version),
                                               colNames.toArray(new String[0])));
       } catch (UnSupportedSArgException e) {
@@ -83,6 +86,7 @@ public class FilterFactory {
                                               Set<String> colIds,
                                               List<PredicateLeaf> leaves,
                                               TypeDescription readSchema,
+                                              boolean isSchemaCaseAware,
                                               OrcFile.Version version)
     throws UnSupportedSArgException {
     VectorFilter result;
@@ -94,6 +98,7 @@ public class FilterFactory {
                                           colIds,
                                           leaves,
                                           readSchema,
+                                          isSchemaCaseAware,
                                           version);
         }
         result = new OrFilter(orFilters);
@@ -105,6 +110,7 @@ public class FilterFactory {
                                            colIds,
                                            leaves,
                                            readSchema,
+                                           isSchemaCaseAware,
                                            version);
         }
         result = new AndFilter(andFilters);
@@ -114,17 +120,19 @@ public class FilterFactory {
         ExpressionTree leaf = expr.getChildren().get(0);
         assert leaf.getOperator() == ExpressionTree.Operator.LEAF;
         result = LeafFilterFactory.createLeafVectorFilter(leaves.get(leaf.getLeaf()),
-                                                             colIds,
-                                                             readSchema,
-                                                             version,
-            true);
+                                                          colIds,
+                                                          readSchema,
+                                                          isSchemaCaseAware,
+                                                          version,
+                                                          true);
         break;
       case LEAF:
         result = LeafFilterFactory.createLeafVectorFilter(leaves.get(expr.getLeaf()),
                                                           colIds,
                                                           readSchema,
+                                                          isSchemaCaseAware,
                                                           version,
-            false);
+                                                          false);
         break;
       default:
         throw new UnSupportedSArgException(String.format("SArg expression: %s is not supported",
diff --git a/java/core/src/java/org/apache/orc/impl/filter/leaf/LeafFilterFactory.java b/java/core/src/java/org/apache/orc/impl/filter/leaf/LeafFilterFactory.java
index c6b3b31..1ff22e7 100644
--- a/java/core/src/java/org/apache/orc/impl/filter/leaf/LeafFilterFactory.java
+++ b/java/core/src/java/org/apache/orc/impl/filter/leaf/LeafFilterFactory.java
@@ -223,11 +223,12 @@ public class LeafFilterFactory {
   public static VectorFilter createLeafVectorFilter(PredicateLeaf leaf,
                                                     Set<String> colIds,
                                                     TypeDescription readSchema,
+                                                    boolean isSchemaCaseAware,
                                                     OrcFile.Version version,
                                                     boolean negated)
       throws FilterFactory.UnSupportedSArgException {
     colIds.add(leaf.getColumnName());
-    TypeDescription colType = readSchema.findSubtype(leaf.getColumnName());
+    TypeDescription colType = readSchema.findSubtype(leaf.getColumnName(), isSchemaCaseAware);
 
     switch (leaf.getOperator()) {
       case IN:
diff --git a/java/core/src/java/org/apache/orc/impl/reader/tree/StructBatchReader.java b/java/core/src/java/org/apache/orc/impl/reader/tree/StructBatchReader.java
index d724e75..ec3c732 100644
--- a/java/core/src/java/org/apache/orc/impl/reader/tree/StructBatchReader.java
+++ b/java/core/src/java/org/apache/orc/impl/reader/tree/StructBatchReader.java
@@ -41,7 +41,9 @@ public class StructBatchReader extends BatchReader {
   public StructBatchReader(TypeReader rowReader, TreeReaderFactory.Context context) {
     super(rowReader);
     this.context = context;
-    this.filterContext = new OrcFilterContextImpl(context.getSchemaEvolution().getReaderSchema());
+    this.filterContext = new OrcFilterContextImpl(context.getSchemaEvolution().getReaderSchema(),
+                                                  context.getSchemaEvolution()
+                                                    .isSchemaEvolutionCaseAware());
     structReader = (TreeReaderFactory.StructTreeReader) rowReader;
   }
 
diff --git a/java/core/src/test/org/apache/orc/TestOrcFilterContext.java b/java/core/src/test/org/apache/orc/TestOrcFilterContext.java
index 2769347..c38e908 100644
--- a/java/core/src/test/org/apache/orc/TestOrcFilterContext.java
+++ b/java/core/src/test/org/apache/orc/TestOrcFilterContext.java
@@ -61,7 +61,7 @@ public class TestOrcFilterContext {
                                            TypeDescription.createList(TypeDescription.createChar()))
                 )
     );
-  private final OrcFilterContext filterContext = new OrcFilterContextImpl(schema)
+  private final OrcFilterContext filterContext = new OrcFilterContextImpl(schema, false)
     .setBatch(schema.createRowBatch());
 
   @BeforeEach
@@ -77,6 +77,13 @@ public class TestOrcFilterContext {
   }
 
   @Test
+  public void testTopLevelElementaryTypeCaseInsensitive() {
+    ColumnVector[] vectorBranch = filterContext.findColumnVector("F1");
+    assertEquals(1, vectorBranch.length);
+    assertTrue(vectorBranch[0] instanceof LongColumnVector);
+  }
+
+  @Test
   public void testTopLevelCompositeType() {
     ColumnVector[] vectorBranch = filterContext.findColumnVector("f3");
     assertEquals(1, vectorBranch.length);
@@ -174,7 +181,7 @@ public class TestOrcFilterContext {
         .addField("a", TypeDescription.createChar())
         .addField("b", TypeDescription
           .createBoolean()));
-    OrcFilterContext fc = new OrcFilterContextImpl(topListSchema)
+    OrcFilterContext fc = new OrcFilterContextImpl(topListSchema, false)
       .setBatch(topListSchema.createRowBatch());
     ColumnVector[] vectorBranch = fc.findColumnVector("_elem");
     assertEquals(2, vectorBranch.length);
diff --git a/java/core/src/test/org/apache/orc/TestRowFilteringIOSkip.java b/java/core/src/test/org/apache/orc/TestRowFilteringIOSkip.java
index a88bb7d..88fd1f9 100644
--- a/java/core/src/test/org/apache/orc/TestRowFilteringIOSkip.java
+++ b/java/core/src/test/org/apache/orc/TestRowFilteringIOSkip.java
@@ -45,6 +45,7 @@ import java.util.function.Consumer;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
 import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
 import static org.junit.jupiter.api.Assertions.assertTrue;
 
 public class TestRowFilteringIOSkip {
@@ -570,6 +571,48 @@ public class TestRowFilteringIOSkip {
     assertEquals(1, rowCount);
   }
 
+  @Test
+  public void readWithCaseSensitivityOff() throws IOException {
+    // Use the ridx column input in UpperCase and flag case-sensitivity off
+    Reader r = OrcFile.createReader(filePath, OrcFile.readerOptions(conf).filesystem(fs));
+    SearchArgument sarg = SearchArgumentFactory.newBuilder()
+      .in("RIDX", PredicateLeaf.Type.LONG, 1L)
+      .build();
+    Reader.Options options = r.options()
+      .searchArgument(sarg, new String[] {"RIDX"})
+      .useSelected(true)
+      .allowSARGToFilter(true)
+      .isSchemaEvolutionCaseAware(false);
+    VectorizedRowBatch b = schema.createRowBatch();
+    long rowCount = 0;
+    try (RecordReader rr = r.rows(options)) {
+      assertTrue(rr.nextBatch(b));
+      validateBatch(b, 1L);
+      rowCount += b.size;
+      assertFalse(rr.nextBatch(b));
+    }
+    assertEquals(1, rowCount);
+  }
+
+  @Test
+  public void readFailureWithCaseSensitivityOn() throws IOException {
+    // Use the ridx column input in UpperCase and flag case-sensitivity off
+    Reader r = OrcFile.createReader(filePath, OrcFile.readerOptions(conf).filesystem(fs));
+    SearchArgument sarg = SearchArgumentFactory.newBuilder()
+      .in("RIDX", PredicateLeaf.Type.LONG, 1L)
+      .build();
+    Reader.Options options = r.options()
+      .searchArgument(sarg, new String[] {"RIDX"})
+      .useSelected(true)
+      .allowSARGToFilter(true)
+      .isSchemaEvolutionCaseAware(true);
+    assertThrows(IllegalArgumentException.class,
+                 () -> r.rows(options),
+                 "Field RIDX not found in struct<f1:bigint,f2:decimal(20,6),f3:bigint,"
+                 + "f4:string,ridx:bigint>");
+
+  }
+
   private void seekToRow(RecordReader rr, VectorizedRowBatch b, long row) throws IOException {
     rr.seekToRow(row);
     assertTrue(rr.nextBatch(b));
diff --git a/java/core/src/test/org/apache/orc/impl/TestOrcFilterContextImpl.java b/java/core/src/test/org/apache/orc/impl/TestOrcFilterContextImpl.java
index def7e58..e585ea7 100644
--- a/java/core/src/test/org/apache/orc/impl/TestOrcFilterContextImpl.java
+++ b/java/core/src/test/org/apache/orc/impl/TestOrcFilterContextImpl.java
@@ -49,7 +49,7 @@ public class TestOrcFilterContextImpl {
   @Test
   public void testSuccessfulRetrieval() {
     VectorizedRowBatch b = createBatch();
-    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
     fc.setBatch(b);
 
     validateF1Vector(fc.findColumnVector("f1"), 1);
@@ -64,7 +64,7 @@ public class TestOrcFilterContextImpl {
     VectorizedRowBatch b1 = createBatch();
     VectorizedRowBatch b2 = createBatch();
     ((LongColumnVector) b2.cols[0]).vector[0] = 100;
-    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
     fc.setBatch(b1);
     validateF1Vector(fc.findColumnVector("f1"), 1);
     // Change the batch
@@ -75,7 +75,7 @@ public class TestOrcFilterContextImpl {
   @Test
   public void testMissingFieldTopLevel() {
     VectorizedRowBatch b = createBatch();
-    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
     fc.setBatch(b);
 
     // Missing field at top level
@@ -87,7 +87,7 @@ public class TestOrcFilterContextImpl {
   @Test
   public void testMissingFieldNestedLevel() {
     VectorizedRowBatch b = createBatch();
-    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
     fc.setBatch(b);
 
     // Missing field at top level
@@ -99,7 +99,7 @@ public class TestOrcFilterContextImpl {
 
   @Test
   public void testPropagations() {
-    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+    OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
     assertNull(fc.getBatch());
     fc.setBatch(schema.createRowBatch());
     assertNotNull(fc.getBatch());
diff --git a/java/core/src/test/org/apache/orc/impl/filter/ATestFilter.java b/java/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
index 551930d..9aadbe8 100644
--- a/java/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
+++ b/java/core/src/test/org/apache/orc/impl/filter/ATestFilter.java
@@ -45,7 +45,7 @@ public class ATestFilter {
     .addField("f3", TypeDescription.createDecimal().withPrecision(38).withScale(2))
     .addField("f4", TypeDescription.createDouble())
     .addField("f5", TypeDescription.createTimestamp());
-  protected final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+  protected final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
 
   protected final VectorizedRowBatch batch = schema.createRowBatch();
 
diff --git a/java/core/src/test/org/apache/orc/impl/filter/TestAndFilter.java b/java/core/src/test/org/apache/orc/impl/filter/TestAndFilter.java
index abd2af0..5ef329a 100644
--- a/java/core/src/test/org/apache/orc/impl/filter/TestAndFilter.java
+++ b/java/core/src/test/org/apache/orc/impl/filter/TestAndFilter.java
@@ -70,6 +70,7 @@ public class TestAndFilter extends ATestFilter {
                                                     colIds,
                                                     sarg.getLeaves(),
                                                     schema,
+                                                    false,
                                                     OrcFile.Version.CURRENT);
     assertNotNull(f);
     assertTrue(f instanceof AndFilter);
diff --git a/java/core/src/test/org/apache/orc/impl/filter/TestConvFilter.java b/java/core/src/test/org/apache/orc/impl/filter/TestConvFilter.java
index cf3b3d4..a85e309 100644
--- a/java/core/src/test/org/apache/orc/impl/filter/TestConvFilter.java
+++ b/java/core/src/test/org/apache/orc/impl/filter/TestConvFilter.java
@@ -41,7 +41,7 @@ public class TestConvFilter {
     .addField("f2", TypeDescription.createDate())
     .addField("f3", TypeDescription.createDecimal().withPrecision(18).withScale(scale));
 
-  private final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema);
+  private final OrcFilterContextImpl fc = new OrcFilterContextImpl(schema, false);
   private final VectorizedRowBatch batch = schema.createRowBatchV2();
 
   @BeforeEach
diff --git a/java/core/src/test/org/apache/orc/impl/filter/TestOrFilter.java b/java/core/src/test/org/apache/orc/impl/filter/TestOrFilter.java
index 66d8168..2c58646 100644
--- a/java/core/src/test/org/apache/orc/impl/filter/TestOrFilter.java
+++ b/java/core/src/test/org/apache/orc/impl/filter/TestOrFilter.java
@@ -46,6 +46,7 @@ public class TestOrFilter extends ATestFilter {
                                                     colIds,
                                                     sarg.getLeaves(),
                                                     schema,
+                                                    false,
                                                     OrcFile.Version.CURRENT);
     assertNotNull(f);
     assertTrue(f instanceof OrFilter);
diff --git a/java/core/src/test/org/apache/orc/impl/filter/leaf/TestFilters.java b/java/core/src/test/org/apache/orc/impl/filter/leaf/TestFilters.java
index cfffa0a..4184f02 100644
--- a/java/core/src/test/org/apache/orc/impl/filter/leaf/TestFilters.java
+++ b/java/core/src/test/org/apache/orc/impl/filter/leaf/TestFilters.java
@@ -60,7 +60,7 @@ public class TestFilters extends ATestFilter {
                                               boolean normalize) {
     Reader.Options options = new Reader.Options().allowSARGToFilter(true);
     options.searchArgument(sArg, new String[0]);
-    return FilterFactory.createBatchFilter(options, readSchema, version, normalize);
+    return FilterFactory.createBatchFilter(options, readSchema, false, version, normalize);
   }
 
   @Test