You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@parquet.apache.org by zi...@apache.org on 2018/10/18 12:08:18 UTC

[parquet-mr] branch master updated: PARQUET-1201: Column indexes (#527)

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

zivanfi pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new e7db9e2  PARQUET-1201: Column indexes (#527)
e7db9e2 is described below

commit e7db9e20f52c925a207ea62d6dda6dc4e870294e
Author: Gabor Szadovszky <ga...@apache.org>
AuthorDate: Thu Oct 18 14:08:13 2018 +0200

    PARQUET-1201: Column indexes (#527)
    
    This is a squashed feature branch merge including the changes listed below. The detailed history can be found in the 'column-indexes' branch.
    
    * PARQUET-1211: Column indexes: read/write API (#456)
    * PARQUET-1212: Column indexes: Show indexes in tools (#479)
    * PARQUET-1213: Column indexes: Limit index size (#480)
    * PARQUET-1214: Column indexes: Truncate min/max values (#481)
    * PARQUET-1364: Invalid row indexes for pages starting with nulls (#507)
    * PARQUET-1310: Column indexes: Filtering (#509)
    * PARQUET-1386: Fix issues of NaN and +-0.0 in case of float/double column indexes (#515)
    * PARQUET-1389: Improve value skipping at page synchronization (#514)
    * PARQUET-1381: Fix missing endRecord after merging columnIndex
---
 .../src/main/java/org/apache/parquet/cli/Main.java |    2 +
 .../cli/commands/ShowColumnIndexCommand.java       |  157 ++
 .../org/apache/parquet/column/ColumnReader.java    |    3 +
 .../apache/parquet/column/ParquetProperties.java   |   20 +-
 .../parquet/column/impl/ColumnReadStoreImpl.java   |   12 +-
 ...ColumnReaderImpl.java => ColumnReaderBase.java} |  122 +-
 .../parquet/column/impl/ColumnReaderImpl.java      |  676 +--------
 ...WriteStoreV2.java => ColumnWriteStoreBase.java} |  112 +-
 .../parquet/column/impl/ColumnWriteStoreV1.java    |  115 +-
 .../parquet/column/impl/ColumnWriteStoreV2.java    |  148 +-
 .../{ColumnWriterV2.java => ColumnWriterBase.java} |  177 +--
 .../apache/parquet/column/impl/ColumnWriterV1.java |  255 +---
 .../apache/parquet/column/impl/ColumnWriterV2.java |  294 +---
 .../column/impl/SynchronizingColumnReader.java     |  111 ++
 .../org/apache/parquet/column/page/DataPage.java   |   22 +
 .../org/apache/parquet/column/page/DataPageV1.java |   31 +
 .../org/apache/parquet/column/page/DataPageV2.java |   52 +
 .../apache/parquet/column/page/PageReadStore.java  |   15 +-
 .../org/apache/parquet/column/page/PageWriter.java |   17 +-
 .../apache/parquet/column/values/ValuesReader.java |   12 +
 .../delta/DeltaBinaryPackingValuesReader.java      |    8 +
 .../DeltaLengthByteArrayValuesReader.java          |   12 +-
 .../plain/FixedLenByteArrayPlainValuesReader.java  |    8 +-
 .../column/values/plain/PlainValuesReader.java     |   36 +-
 .../rle/RunLengthBitPackingHybridValuesWriter.java |    9 +-
 .../column/values/rle/ZeroIntegerValuesReader.java |    4 +
 .../columnindex/BinaryColumnIndexBuilder.java      |  140 ++
 .../column/columnindex/BinaryTruncator.java        |  208 +++
 .../columnindex/BooleanColumnIndexBuilder.java     |  133 ++
 .../internal/column/columnindex/BoundaryOrder.java |  352 +++++
 .../internal/column/columnindex/ColumnIndex.java   |   60 +
 .../column/columnindex/ColumnIndexBuilder.java     |  636 ++++++++
 .../columnindex/DoubleColumnIndexBuilder.java      |  155 ++
 .../columnindex/FloatColumnIndexBuilder.java       |  155 ++
 .../internal/column/columnindex/IndexIterator.java |   98 ++
 .../column/columnindex/IntColumnIndexBuilder.java  |  136 ++
 .../column/columnindex/LongColumnIndexBuilder.java |  136 ++
 .../internal/column/columnindex/OffsetIndex.java   |   64 +
 .../column/columnindex/OffsetIndexBuilder.java     |  175 +++
 .../filter2/columnindex/ColumnIndexFilter.java     |  194 +++
 .../filter2/columnindex/ColumnIndexStore.java      |   55 +
 .../internal/filter2/columnindex/RowRanges.java    |  288 ++++
 .../parquet/column/impl/TestColumnReaderImpl.java  |    8 +-
 .../apache/parquet/column/mem/TestMemColumn.java   |   12 +-
 .../parquet/column/page/mem/MemPageWriter.java     |    6 +
 .../values/bitpacking/TestBitPackingColumn.java    |   16 +
 ...ltaBinaryPackingValuesWriterForIntegerTest.java |   17 +
 .../DeltaBinaryPackingValuesWriterForLongTest.java |   17 +
 .../TestDeltaLengthByteArray.java                  |   24 +
 .../values/deltastrings/TestDeltaByteArray.java    |   19 +
 .../column/values/dictionary/TestDictionary.java   |  105 ++
 .../column/columnindex/TestBinaryTruncator.java    |  285 ++++
 .../column/columnindex/TestBoundaryOrder.java      |  487 ++++++
 .../column/columnindex/TestColumnIndexBuilder.java | 1546 ++++++++++++++++++++
 .../column/columnindex/TestIndexIterator.java      |   63 +
 .../column/columnindex/TestOffsetIndexBuilder.java |  113 ++
 .../filter2/columnindex/TestColumnIndexFilter.java |  464 ++++++
 .../filter2/columnindex/TestRowRanges.java         |  155 ++
 parquet-common/pom.xml                             |    6 +
 .../java/org/apache/parquet/HadoopReadOptions.java |    9 +-
 .../org/apache/parquet/ParquetReadOptions.java     |   20 +-
 .../format/converter/ParquetMetadataConverter.java |  108 ++
 .../parquet/hadoop/ColumnChunkPageReadStore.java   |  115 +-
 .../parquet/hadoop/ColumnChunkPageWriteStore.java  |   54 +-
 .../parquet/hadoop/ColumnIndexFilterUtils.java     |  157 ++
 .../parquet/hadoop/ColumnIndexStoreImpl.java       |  155 ++
 .../hadoop/InternalParquetRecordReader.java        |    6 +-
 .../hadoop/InternalParquetRecordWriter.java        |    4 +-
 .../apache/parquet/hadoop/ParquetFileReader.java   |  335 ++++-
 .../apache/parquet/hadoop/ParquetFileWriter.java   |  222 ++-
 .../apache/parquet/hadoop/ParquetInputFormat.java  |    5 +
 .../apache/parquet/hadoop/ParquetOutputFormat.java |   17 +-
 .../org/apache/parquet/hadoop/ParquetReader.java   |   10 +
 .../org/apache/parquet/hadoop/ParquetWriter.java   |    2 +-
 .../hadoop/metadata/ColumnChunkMetaData.java       |   41 +-
 .../internal/hadoop/metadata/IndexReference.java   |   40 +-
 .../filter2/recordlevel/PhoneBookWriter.java       |  105 +-
 .../converter/TestParquetMetadataConverter.java    |   62 +
 .../hadoop/TestColumnChunkPageWriteStore.java      |   94 +-
 .../parquet/hadoop/TestColumnIndexFiltering.java   |  442 ++++++
 .../parquet/hadoop/TestParquetFileWriter.java      |  146 ++
 .../parquet/tools/command/ColumnIndexCommand.java  |  182 +++
 .../org/apache/parquet/tools/command/Registry.java |    1 +
 83 files changed, 9368 insertions(+), 1722 deletions(-)

diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
index 990193c..fa69ce7 100644
--- a/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/Main.java
@@ -32,6 +32,7 @@ import org.apache.parquet.cli.commands.ConvertCSVCommand;
 import org.apache.parquet.cli.commands.ConvertCommand;
 import org.apache.parquet.cli.commands.ParquetMetadataCommand;
 import org.apache.parquet.cli.commands.SchemaCommand;
+import org.apache.parquet.cli.commands.ShowColumnIndexCommand;
 import org.apache.parquet.cli.commands.ShowDictionaryCommand;
 import org.apache.parquet.cli.commands.ShowPagesCommand;
 import org.apache.parquet.cli.commands.ToAvroCommand;
@@ -87,6 +88,7 @@ public class Main extends Configured implements Tool {
     jc.addCommand("to-avro", new ToAvroCommand(console));
     jc.addCommand("cat", new CatCommand(console, 0));
     jc.addCommand("head", new CatCommand(console, 10));
+    jc.addCommand("column-index", new ShowColumnIndexCommand(console));
   }
 
   @Override
diff --git a/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowColumnIndexCommand.java b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowColumnIndexCommand.java
new file mode 100644
index 0000000..38a7094
--- /dev/null
+++ b/parquet-cli/src/main/java/org/apache/parquet/cli/commands/ShowColumnIndexCommand.java
@@ -0,0 +1,157 @@
+/*
+ * 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.parquet.cli.commands;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.parquet.cli.BaseCommand;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.io.InputFile;
+import org.slf4j.Logger;
+
+import com.beust.jcommander.Parameter;
+import com.beust.jcommander.Parameters;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+/**
+ * parquet-cli command to print column and offset indexes.
+ */
+@Parameters(commandDescription = "Prints the column and offset indexes of a Parquet file")
+public class ShowColumnIndexCommand extends BaseCommand {
+  public ShowColumnIndexCommand(Logger console) {
+    super(console);
+  }
+
+  @Parameter(description = "<parquet path>")
+  List<String> files;
+
+  @Parameter(names = { "-c", "--column" }, description = "Shows the column/offset indexes for the given column only")
+  List<String> ColumnPaths;
+
+  @Parameter(names = { "-r",
+      "--row-group" }, description = "Shows the column/offset indexes for the given row-groups only; "
+          + "row-groups are referenced by their indexes from 0")
+  List<String> rowGroupIndexes;
+
+  @Parameter(names = { "-i", "--column-index" }, description = "Shows the column indexes; "
+      + "active by default unless -o is used")
+  boolean showColumnIndex;
+
+  @Parameter(names = { "-o", "--offset-index" }, description = "Shows the offset indexes; "
+      + "active by default unless -i is used")
+  boolean showOffsetIndex;
+
+  @Override
+  public List<String> getExamples() {
+    return Lists.newArrayList(
+        "# Show only column indexes for column 'col' from a Parquet file",
+        "-c col -i sample.parquet");
+  }
+
+  @Override
+  public int run() throws IOException {
+    Preconditions.checkArgument(files != null && files.size() >= 1,
+        "A Parquet file is required.");
+    Preconditions.checkArgument(files.size() == 1,
+        "Cannot process multiple Parquet files.");
+
+    InputFile in = HadoopInputFile.fromPath(qualifiedPath(files.get(0)), getConf());
+    if (!showColumnIndex && !showOffsetIndex) {
+      showColumnIndex = true;
+      showOffsetIndex = true;
+    }
+
+    Set<String> rowGroupIndexSet = new HashSet<>();
+    if (rowGroupIndexes != null) {
+      rowGroupIndexSet.addAll(rowGroupIndexes);
+    }
+
+    try (ParquetFileReader reader = ParquetFileReader.open(in)) {
+      boolean firstBlock = true;
+      int rowGroupIndex = 0;
+      for (BlockMetaData block : reader.getFooter().getBlocks()) {
+        if (!rowGroupIndexSet.isEmpty() && !rowGroupIndexSet.contains(Integer.toString(rowGroupIndex))) {
+          ++rowGroupIndex;
+          continue;
+        }
+        if (!firstBlock) {
+          console.info("");
+        }
+        firstBlock = false;
+        console.info("row-group {}:", rowGroupIndex);
+        for (ColumnChunkMetaData column : getColumns(block)) {
+          String path = column.getPath().toDotString();
+          if (showColumnIndex) {
+            console.info("column index for column {}:", path);
+            ColumnIndex columnIndex = reader.readColumnIndex(column);
+            if (columnIndex == null) {
+              console.info("NONE");
+            } else {
+              console.info(columnIndex.toString());
+            }
+          }
+          if (showOffsetIndex) {
+            console.info("offset index for column {}:", path);
+            OffsetIndex offsetIndex = reader.readOffsetIndex(column);
+            if (offsetIndex == null) {
+              console.info("NONE");
+            } else {
+              console.info(offsetIndex.toString());
+            }
+          }
+        }
+        ++rowGroupIndex;
+      }
+    }
+    return 0;
+  }
+
+  private List<ColumnChunkMetaData> getColumns(BlockMetaData block) {
+    List<ColumnChunkMetaData> columns = block.getColumns();
+    if (ColumnPaths == null || ColumnPaths.isEmpty()) {
+      return columns;
+    }
+    Map<String, ColumnChunkMetaData> pathMap = new HashMap<>();
+    for (ColumnChunkMetaData column : columns) {
+      pathMap.put(column.getPath().toDotString(), column);
+    }
+
+    List<ColumnChunkMetaData> filtered = new ArrayList<>();
+    for (String path : ColumnPaths) {
+      ColumnChunkMetaData column = pathMap.get(path);
+      if (column != null) {
+        filtered.add(column);
+      }
+    }
+    return filtered;
+  }
+
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ColumnReader.java b/parquet-column/src/main/java/org/apache/parquet/column/ColumnReader.java
index 52d269e..6d93eee 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ColumnReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ColumnReader.java
@@ -41,7 +41,10 @@ public interface ColumnReader {
 
   /**
    * @return the totalCount of values to be consumed
+   * @deprecated will be removed in 2.0.0; Total values might not be able to be counted before reading the values (e.g.
+   *             in case of column index based filtering)
    */
+  @Deprecated
   long getTotalValueCount();
 
   /**
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
index 39b65da..b173239 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/ParquetProperties.java
@@ -47,6 +47,7 @@ public class ParquetProperties {
   public static final boolean DEFAULT_ESTIMATE_ROW_COUNT_FOR_PAGE_SIZE_CHECK = true;
   public static final int DEFAULT_MINIMUM_RECORD_COUNT_FOR_CHECK = 100;
   public static final int DEFAULT_MAXIMUM_RECORD_COUNT_FOR_CHECK = 10000;
+  public static final int DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH = 64;
 
   public static final ValuesWriterFactory DEFAULT_VALUES_WRITER_FACTORY = new DefaultValuesWriterFactory();
 
@@ -83,10 +84,11 @@ public class ParquetProperties {
   private final boolean estimateNextSizeCheck;
   private final ByteBufferAllocator allocator;
   private final ValuesWriterFactory valuesWriterFactory;
+  private final int columnIndexTruncateLength;
 
   private ParquetProperties(WriterVersion writerVersion, int pageSize, int dictPageSize, boolean enableDict, int minRowCountForPageSizeCheck,
                             int maxRowCountForPageSizeCheck, boolean estimateNextSizeCheck, ByteBufferAllocator allocator,
-                            ValuesWriterFactory writerFactory) {
+                            ValuesWriterFactory writerFactory, int columnIndexMinMaxTruncateLength) {
     this.pageSizeThreshold = pageSize;
     this.initialSlabSize = CapacityByteArrayOutputStream
       .initialSlabSizeHeuristic(MIN_SLAB_SIZE, pageSizeThreshold, 10);
@@ -99,6 +101,7 @@ public class ParquetProperties {
     this.allocator = allocator;
 
     this.valuesWriterFactory = writerFactory;
+    this.columnIndexTruncateLength = columnIndexMinMaxTruncateLength;
   }
 
   public ValuesWriter newRepetitionLevelWriter(ColumnDescriptor path) {
@@ -163,7 +166,7 @@ public class ParquetProperties {
                                               PageWriteStore pageStore) {
     switch (writerVersion) {
     case PARQUET_1_0:
-      return new ColumnWriteStoreV1(pageStore, this);
+      return new ColumnWriteStoreV1(schema, pageStore, this);
     case PARQUET_2_0:
       return new ColumnWriteStoreV2(schema, pageStore, this);
     default:
@@ -183,6 +186,10 @@ public class ParquetProperties {
     return valuesWriterFactory;
   }
 
+  public int getColumnIndexTruncateLength() {
+    return columnIndexTruncateLength;
+  }
+
   public boolean estimateNextSizeCheck() {
     return estimateNextSizeCheck;
   }
@@ -205,6 +212,7 @@ public class ParquetProperties {
     private boolean estimateNextSizeCheck = DEFAULT_ESTIMATE_ROW_COUNT_FOR_PAGE_SIZE_CHECK;
     private ByteBufferAllocator allocator = new HeapByteBufferAllocator();
     private ValuesWriterFactory valuesWriterFactory = DEFAULT_VALUES_WRITER_FACTORY;
+    private int columnIndexTruncateLength = DEFAULT_COLUMN_INDEX_TRUNCATE_LENGTH;
 
     private Builder() {
     }
@@ -299,11 +307,17 @@ public class ParquetProperties {
       return this;
     }
 
+    public Builder withColumnIndexTruncateLength(int length) {
+      Preconditions.checkArgument(length > 0, "Invalid column index min/max truncate length (negative) : %s", length);
+      this.columnIndexTruncateLength = length;
+      return this;
+    }
+
     public ParquetProperties build() {
       ParquetProperties properties =
         new ParquetProperties(writerVersion, pageSize, dictPageSize,
           enableDict, minRowCountForPageSizeCheck, maxRowCountForPageSizeCheck,
-          estimateNextSizeCheck, allocator, valuesWriterFactory);
+          estimateNextSizeCheck, allocator, valuesWriterFactory, columnIndexTruncateLength);
       // we pass a constructed but uninitialized factory to ParquetProperties above as currently
       // creation of ValuesWriters is invoked from within ParquetProperties. In the future
       // we'd like to decouple that and won't need to pass an object to properties and then pass the
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReadStoreImpl.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReadStoreImpl.java
index e582908..b7e1597 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReadStoreImpl.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReadStoreImpl.java
@@ -18,6 +18,9 @@
  */
 package org.apache.parquet.column.impl;
 
+import java.util.Optional;
+import java.util.PrimitiveIterator;
+
 import org.apache.parquet.VersionParser;
 import org.apache.parquet.VersionParser.ParsedVersion;
 import org.apache.parquet.VersionParser.VersionParseException;
@@ -72,7 +75,14 @@ public class ColumnReadStoreImpl implements ColumnReadStore {
 
   @Override
   public ColumnReader getColumnReader(ColumnDescriptor path) {
-    return newMemColumnReader(path, pageReadStore.getPageReader(path));
+    PrimitiveConverter converter = getPrimitiveConverter(path);
+    PageReader pageReader = pageReadStore.getPageReader(path);
+    Optional<PrimitiveIterator.OfLong> rowIndexes = pageReadStore.getRowIndexes();
+    if (rowIndexes.isPresent()) {
+      return new SynchronizingColumnReader(path, pageReader, converter, writerVersion, rowIndexes.get());
+    } else {
+      return new ColumnReaderImpl(path, pageReader, converter, writerVersion);
+    }
   }
 
   public ColumnReaderImpl newMemColumnReader(ColumnDescriptor path, PageReader pageReader) {
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderBase.java
similarity index 87%
copy from parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
copy to parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderBase.java
index 8c85b37..c929431 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderBase.java
@@ -52,10 +52,10 @@ import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * ColumnReader implementation
+ * Base superclass for {@link ColumnReader} implementations.
  */
-public class ColumnReaderImpl implements ColumnReader {
-  private static final Logger LOG = LoggerFactory.getLogger(ColumnReaderImpl.class);
+abstract class ColumnReaderBase implements ColumnReader {
+  private static final Logger LOG = LoggerFactory.getLogger(ColumnReaderBase.class);
 
   /**
    * binds the lower level page decoder to the record converter materializing the records
@@ -73,6 +73,14 @@ public class ColumnReaderImpl implements ColumnReader {
     abstract void skip();
 
     /**
+     * Skips n values from the underlying page
+     *
+     * @param n
+     *          the number of values to be skipped
+     */
+    abstract void skip(int n);
+
+    /**
      * write current value to converter
      */
     abstract void writeValue();
@@ -148,6 +156,7 @@ public class ColumnReaderImpl implements ColumnReader {
 
   private final PrimitiveConverter converter;
   private Binding binding;
+  private final int maxDefinitionLevel;
 
   // this is needed because we will attempt to read the value twice when filtering
   // TODO: rework that
@@ -162,6 +171,10 @@ public class ColumnReaderImpl implements ColumnReader {
           public void skip() {
             dataColumn.skip();
           }
+          @Override
+          void skip(int n) {
+            dataColumn.skip(n);
+          }
           public int getDictionaryId() {
             return dictionaryId;
           }
@@ -202,6 +215,11 @@ public class ColumnReaderImpl implements ColumnReader {
             current = 0;
             dataColumn.skip();
           }
+          @Override
+          void skip(int n) {
+            current = 0;
+            dataColumn.skip(n);
+          }
           public float getFloat() {
             return current;
           }
@@ -221,6 +239,11 @@ public class ColumnReaderImpl implements ColumnReader {
             current = 0;
             dataColumn.skip();
           }
+          @Override
+          void skip(int n) {
+            current = 0;
+            dataColumn.skip(n);
+          }
           public double getDouble() {
             return current;
           }
@@ -241,6 +264,11 @@ public class ColumnReaderImpl implements ColumnReader {
             dataColumn.skip();
           }
           @Override
+          void skip(int n) {
+            current = 0;
+            dataColumn.skip(n);
+          }
+          @Override
           public int getInteger() {
             return current;
           }
@@ -261,6 +289,11 @@ public class ColumnReaderImpl implements ColumnReader {
             dataColumn.skip();
           }
           @Override
+          void skip(int n) {
+            current = 0;
+            dataColumn.skip(n);
+          }
+          @Override
           public long getLong() {
             return current;
           }
@@ -290,6 +323,11 @@ public class ColumnReaderImpl implements ColumnReader {
             dataColumn.skip();
           }
           @Override
+          void skip(int n) {
+            current = false;
+            dataColumn.skip(n);
+          }
+          @Override
           public boolean getBoolean() {
             return current;
           }
@@ -310,6 +348,11 @@ public class ColumnReaderImpl implements ColumnReader {
             dataColumn.skip();
           }
           @Override
+          void skip(int n) {
+            current = null;
+            dataColumn.skip(n);
+          }
+          @Override
           public Binary getBinary() {
             return current;
           }
@@ -328,11 +371,12 @@ public class ColumnReaderImpl implements ColumnReader {
    * @param converter a converter that materializes the values in this column in the current record
    * @param writerVersion writer version string from the Parquet file being read
    */
-  public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter, ParsedVersion writerVersion) {
+  ColumnReaderBase(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter, ParsedVersion writerVersion) {
     this.path = checkNotNull(path, "path");
     this.pageReader = checkNotNull(pageReader, "pageReader");
     this.converter = checkNotNull(converter, "converter");
     this.writerVersion = writerVersion;
+    this.maxDefinitionLevel = path.getMaxDefinitionLevel();
     DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
     if (dictionaryPage != null) {
       try {
@@ -350,10 +394,9 @@ public class ColumnReaderImpl implements ColumnReader {
     if (totalValueCount <= 0) {
       throw new ParquetDecodingException("totalValueCount '" + totalValueCount + "' <= 0");
     }
-    consume();
   }
 
-  private boolean isFullyConsumed() {
+  boolean isFullyConsumed() {
     return readValues >= totalValueCount;
   }
 
@@ -508,25 +551,39 @@ public class ColumnReaderImpl implements ColumnReader {
     return definitionLevel;
   }
 
-  // TODO: change the logic around read() to not tie together reading from the 3 columns
-  private void readRepetitionAndDefinitionLevels() {
-    repetitionLevel = repetitionLevelColumn.nextInt();
-    definitionLevel = definitionLevelColumn.nextInt();
-    ++readValues;
-  }
-
   private void checkRead() {
-    if (isPageFullyConsumed()) {
-      if (isFullyConsumed()) {
-        LOG.debug("end reached");
-        repetitionLevel = 0; // the next repetition level
-        return;
+    int rl, dl;
+    int skipValues = 0;
+    for (;;) {
+      if (isPageFullyConsumed()) {
+        if (isFullyConsumed()) {
+          LOG.debug("end reached");
+          repetitionLevel = 0; // the next repetition level
+          return;
+        }
+        readPage();
+        skipValues = 0;
+      }
+      rl = repetitionLevelColumn.nextInt();
+      dl = definitionLevelColumn.nextInt();
+      ++readValues;
+      if (!skipRL(rl)) {
+        break;
+      }
+      if (dl == maxDefinitionLevel) {
+        ++skipValues;
       }
-      readPage();
     }
-    readRepetitionAndDefinitionLevels();
+    binding.skip(skipValues);
+    repetitionLevel = rl;
+    definitionLevel = dl;
   }
 
+  /*
+   * Returns if current levels / value shall be skipped based on the specified repetition level.
+   */
+  abstract boolean skipRL(int rl);
+
   private void readPage() {
     LOG.debug("loading page");
     DataPage page = pageReader.readPage();
@@ -585,32 +642,42 @@ public class ColumnReaderImpl implements ColumnReader {
     ValuesReader dlReader = page.getDlEncoding().getValuesReader(path, DEFINITION_LEVEL);
     this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader);
     this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader);
+    int valueCount = page.getValueCount();
     try {
       BytesInput bytes = page.getBytes();
-      LOG.debug("page size {} bytes and {} records", bytes.size(), pageValueCount);
+      LOG.debug("page size {} bytes and {} values", bytes.size(), valueCount);
       LOG.debug("reading repetition levels at 0");
       ByteBufferInputStream in = bytes.toInputStream();
-      rlReader.initFromPage(pageValueCount, in);
+      rlReader.initFromPage(valueCount, in);
       LOG.debug("reading definition levels at {}", in.position());
-      dlReader.initFromPage(pageValueCount, in);
+      dlReader.initFromPage(valueCount, in);
       LOG.debug("reading data at {}", in.position());
-      initDataReader(page.getValueEncoding(), in, page.getValueCount());
+      initDataReader(page.getValueEncoding(), in, valueCount);
     } catch (IOException e) {
       throw new ParquetDecodingException("could not read page " + page + " in col " + path, e);
     }
+    newPageInitialized(page);
   }
 
   private void readPageV2(DataPageV2 page) {
     this.repetitionLevelColumn = newRLEIterator(path.getMaxRepetitionLevel(), page.getRepetitionLevels());
     this.definitionLevelColumn = newRLEIterator(path.getMaxDefinitionLevel(), page.getDefinitionLevels());
-    LOG.debug("page data size {} bytes and {} records", page.getData().size(), pageValueCount);
+    int valueCount = page.getValueCount();
+    LOG.debug("page data size {} bytes and {} values", page.getData().size(), valueCount);
     try {
-      initDataReader(page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount());
+      initDataReader(page.getDataEncoding(), page.getData().toInputStream(), valueCount);
     } catch (IOException e) {
       throw new ParquetDecodingException("could not read page " + page + " in col " + path, e);
     }
+    newPageInitialized(page);
+  }
+
+  final int getPageValueCount() {
+    return pageValueCount;
   }
 
+  abstract void newPageInitialized(DataPage page);
+
   private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
     try {
       if (maxLevel == 0) {
@@ -625,7 +692,7 @@ public class ColumnReaderImpl implements ColumnReader {
     }
   }
 
-  private boolean isPageFullyConsumed() {
+  boolean isPageFullyConsumed() {
     return readValues >= endOfPageValueCount;
   }
 
@@ -643,6 +710,7 @@ public class ColumnReaderImpl implements ColumnReader {
    * {@inheritDoc}
    * @see org.apache.parquet.column.ColumnReader#getTotalValueCount()
    */
+  @Deprecated
   @Override
   public long getTotalValueCount() {
     return totalValueCount;
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
index 8c85b37..0413d62 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnReaderImpl.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -18,675 +18,41 @@
  */
 package org.apache.parquet.column.impl;
 
-import static java.lang.String.format;
-import static org.apache.parquet.Preconditions.checkNotNull;
-import static org.apache.parquet.column.ValuesType.DEFINITION_LEVEL;
-import static org.apache.parquet.column.ValuesType.REPETITION_LEVEL;
-import static org.apache.parquet.column.ValuesType.VALUES;
-
-import java.io.IOException;
-
-import org.apache.parquet.CorruptDeltaByteArrays;
 import org.apache.parquet.VersionParser.ParsedVersion;
-import org.apache.parquet.bytes.ByteBufferInputStream;
-import org.apache.parquet.bytes.BytesInput;
-import org.apache.parquet.bytes.BytesUtils;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.ColumnReader;
-import org.apache.parquet.column.Dictionary;
-import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.page.DataPage;
-import org.apache.parquet.column.page.DataPageV1;
-import org.apache.parquet.column.page.DataPageV2;
-import org.apache.parquet.column.page.DictionaryPage;
 import org.apache.parquet.column.page.PageReader;
-import org.apache.parquet.column.values.RequiresPreviousReader;
-import org.apache.parquet.column.values.ValuesReader;
-import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridDecoder;
-import org.apache.parquet.io.ParquetDecodingException;
-import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.io.api.PrimitiveConverter;
-import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
-import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeNameConverter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
- * ColumnReader implementation
+ * ColumnReader implementation for the scenario when column indexes are not used (all values are read)
  */
-public class ColumnReaderImpl implements ColumnReader {
-  private static final Logger LOG = LoggerFactory.getLogger(ColumnReaderImpl.class);
-
-  /**
-   * binds the lower level page decoder to the record converter materializing the records
-   */
-  private static abstract class Binding {
-
-    /**
-     * read one value from the underlying page
-     */
-    abstract void read();
-
-    /**
-     * skip one value from the underlying page
-     */
-    abstract void skip();
-
-    /**
-     * write current value to converter
-     */
-    abstract void writeValue();
-
-    /**
-     * @return current value
-     */
-    public int getDictionaryId() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public int getInteger() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public boolean getBoolean() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public long getLong() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public Binary getBinary() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public float getFloat() {
-      throw new UnsupportedOperationException();
-    }
-
-    /**
-     * @return current value
-     */
-    public double getDouble() {
-      throw new UnsupportedOperationException();
-    }
-  }
-
-  private final ParsedVersion writerVersion;
-  private final ColumnDescriptor path;
-  private final long totalValueCount;
-  private final PageReader pageReader;
-  private final Dictionary dictionary;
-
-  private IntIterator repetitionLevelColumn;
-  private IntIterator definitionLevelColumn;
-  protected ValuesReader dataColumn;
-  private Encoding currentEncoding;
-
-  private int repetitionLevel;
-  private int definitionLevel;
-  private int dictionaryId;
-
-  private long endOfPageValueCount;
-  private long readValues = 0;
-  private int pageValueCount = 0;
-
-  private final PrimitiveConverter converter;
-  private Binding binding;
-
-  // this is needed because we will attempt to read the value twice when filtering
-  // TODO: rework that
-  private boolean valueRead;
-
-  private void bindToDictionary(final Dictionary dictionary) {
-    binding =
-        new Binding() {
-          void read() {
-            dictionaryId = dataColumn.readValueDictionaryId();
-          }
-          public void skip() {
-            dataColumn.skip();
-          }
-          public int getDictionaryId() {
-            return dictionaryId;
-          }
-          void writeValue() {
-            converter.addValueFromDictionary(dictionaryId);
-          }
-          public int getInteger() {
-            return dictionary.decodeToInt(dictionaryId);
-          }
-          public boolean getBoolean() {
-            return dictionary.decodeToBoolean(dictionaryId);
-          }
-          public long getLong() {
-            return dictionary.decodeToLong(dictionaryId);
-          }
-          public Binary getBinary() {
-            return dictionary.decodeToBinary(dictionaryId);
-          }
-          public float getFloat() {
-            return dictionary.decodeToFloat(dictionaryId);
-          }
-          public double getDouble() {
-            return dictionary.decodeToDouble(dictionaryId);
-          }
-        };
-  }
-
-  private void bind(PrimitiveTypeName type) {
-    binding = type.convert(new PrimitiveTypeNameConverter<Binding, RuntimeException>() {
-      @Override
-      public Binding convertFLOAT(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          float current;
-          void read() {
-            current = dataColumn.readFloat();
-          }
-          public void skip() {
-            current = 0;
-            dataColumn.skip();
-          }
-          public float getFloat() {
-            return current;
-          }
-          void writeValue() {
-            converter.addFloat(current);
-          }
-        };
-      }
-      @Override
-      public Binding convertDOUBLE(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          double current;
-          void read() {
-            current = dataColumn.readDouble();
-          }
-          public void skip() {
-            current = 0;
-            dataColumn.skip();
-          }
-          public double getDouble() {
-            return current;
-          }
-          void writeValue() {
-            converter.addDouble(current);
-          }
-        };
-      }
-      @Override
-      public Binding convertINT32(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          int current;
-          void read() {
-            current = dataColumn.readInteger();
-          }
-          public void skip() {
-            current = 0;
-            dataColumn.skip();
-          }
-          @Override
-          public int getInteger() {
-            return current;
-          }
-          void writeValue() {
-            converter.addInt(current);
-          }
-        };
-      }
-      @Override
-      public Binding convertINT64(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          long current;
-          void read() {
-            current = dataColumn.readLong();
-          }
-          public void skip() {
-            current = 0;
-            dataColumn.skip();
-          }
-          @Override
-          public long getLong() {
-            return current;
-          }
-          void writeValue() {
-            converter.addLong(current);
-          }
-        };
-      }
-      @Override
-      public Binding convertINT96(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return this.convertBINARY(primitiveTypeName);
-      }
-      @Override
-      public Binding convertFIXED_LEN_BYTE_ARRAY(
-          PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return this.convertBINARY(primitiveTypeName);
-      }
-      @Override
-      public Binding convertBOOLEAN(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          boolean current;
-          void read() {
-            current = dataColumn.readBoolean();
-          }
-          public void skip() {
-            current = false;
-            dataColumn.skip();
-          }
-          @Override
-          public boolean getBoolean() {
-            return current;
-          }
-          void writeValue() {
-            converter.addBoolean(current);
-          }
-        };
-      }
-      @Override
-      public Binding convertBINARY(PrimitiveTypeName primitiveTypeName) throws RuntimeException {
-        return new Binding() {
-          Binary current;
-          void read() {
-            current = dataColumn.readBytes();
-          }
-          public void skip() {
-            current = null;
-            dataColumn.skip();
-          }
-          @Override
-          public Binary getBinary() {
-            return current;
-          }
-          void writeValue() {
-            converter.addBinary(current);
-          }
-        };
-      }
-    });
-  }
+public class ColumnReaderImpl extends ColumnReaderBase {
 
   /**
    * creates a reader for triplets
-   * @param path the descriptor for the corresponding column
-   * @param pageReader the underlying store to read from
-   * @param converter a converter that materializes the values in this column in the current record
-   * @param writerVersion writer version string from the Parquet file being read
-   */
-  public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter, ParsedVersion writerVersion) {
-    this.path = checkNotNull(path, "path");
-    this.pageReader = checkNotNull(pageReader, "pageReader");
-    this.converter = checkNotNull(converter, "converter");
-    this.writerVersion = writerVersion;
-    DictionaryPage dictionaryPage = pageReader.readDictionaryPage();
-    if (dictionaryPage != null) {
-      try {
-        this.dictionary = dictionaryPage.getEncoding().initDictionary(path, dictionaryPage);
-        if (converter.hasDictionarySupport()) {
-          converter.setDictionary(dictionary);
-        }
-      } catch (IOException e) {
-        throw new ParquetDecodingException("could not decode the dictionary for " + path, e);
-      }
-    } else {
-      this.dictionary = null;
-    }
-    this.totalValueCount = pageReader.getTotalValueCount();
-    if (totalValueCount <= 0) {
-      throw new ParquetDecodingException("totalValueCount '" + totalValueCount + "' <= 0");
-    }
+   * 
+   * @param path
+   *          the descriptor for the corresponding column
+   * @param pageReader
+   *          the underlying store to read from
+   * @param converter
+   *          a converter that materializes the values in this column in the current record
+   * @param writerVersion
+   *          writer version string from the Parquet file being read
+   */
+  public ColumnReaderImpl(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter,
+      ParsedVersion writerVersion) {
+    super(path, pageReader, converter, writerVersion);
     consume();
   }
 
-  private boolean isFullyConsumed() {
-    return readValues >= totalValueCount;
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#writeCurrentValueToConverter()
-   */
-  @Override
-  public void writeCurrentValueToConverter() {
-    readValue();
-    this.binding.writeValue();
-  }
-
-  @Override
-  public int getCurrentValueDictionaryID() {
-    readValue();
-    return binding.getDictionaryId();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getInteger()
-   */
-  @Override
-  public int getInteger() {
-    readValue();
-    return this.binding.getInteger();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getBoolean()
-   */
-  @Override
-  public boolean getBoolean() {
-    readValue();
-    return this.binding.getBoolean();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getLong()
-   */
-  @Override
-  public long getLong() {
-    readValue();
-    return this.binding.getLong();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getBinary()
-   */
-  @Override
-  public Binary getBinary() {
-    readValue();
-    return this.binding.getBinary();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getFloat()
-   */
-  @Override
-  public float getFloat() {
-    readValue();
-    return this.binding.getFloat();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getDouble()
-   */
-  @Override
-  public double getDouble() {
-    readValue();
-    return this.binding.getDouble();
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getCurrentRepetitionLevel()
-   */
-  @Override
-  public int getCurrentRepetitionLevel() {
-    return repetitionLevel;
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getDescriptor()
-   */
-  @Override
-  public ColumnDescriptor getDescriptor() {
-    return path;
-  }
-
-  /**
-   * Reads the value into the binding.
-   */
-  public void readValue() {
-    try {
-      if (!valueRead) {
-        binding.read();
-        valueRead = true;
-      }
-    } catch (RuntimeException e) {
-      if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, currentEncoding) &&
-          e instanceof ArrayIndexOutOfBoundsException) {
-        // this is probably PARQUET-246, which may happen if reading data with
-        // MR because this can't be detected without reading all footers
-        throw new ParquetDecodingException("Read failure possibly due to " +
-            "PARQUET-246: try setting parquet.split.files to false",
-            new ParquetDecodingException(
-                format("Can't read value in column %s at value %d out of %d, " +
-                        "%d out of %d in currentPage. repetition level: " +
-                        "%d, definition level: %d",
-                    path, readValues, totalValueCount,
-                    readValues - (endOfPageValueCount - pageValueCount),
-                    pageValueCount, repetitionLevel, definitionLevel),
-                e));
-      }
-      throw new ParquetDecodingException(
-          format("Can't read value in column %s at value %d out of %d, " +
-                  "%d out of %d in currentPage. repetition level: " +
-                  "%d, definition level: %d",
-              path, readValues, totalValueCount,
-              readValues - (endOfPageValueCount - pageValueCount),
-              pageValueCount, repetitionLevel, definitionLevel),
-          e);
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#skip()
-   */
-  @Override
-  public void skip() {
-    if (!valueRead) {
-      binding.skip();
-      valueRead = true;
-    }
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getCurrentDefinitionLevel()
-   */
-  @Override
-  public int getCurrentDefinitionLevel() {
-    return definitionLevel;
-  }
-
-  // TODO: change the logic around read() to not tie together reading from the 3 columns
-  private void readRepetitionAndDefinitionLevels() {
-    repetitionLevel = repetitionLevelColumn.nextInt();
-    definitionLevel = definitionLevelColumn.nextInt();
-    ++readValues;
-  }
-
-  private void checkRead() {
-    if (isPageFullyConsumed()) {
-      if (isFullyConsumed()) {
-        LOG.debug("end reached");
-        repetitionLevel = 0; // the next repetition level
-        return;
-      }
-      readPage();
-    }
-    readRepetitionAndDefinitionLevels();
-  }
-
-  private void readPage() {
-    LOG.debug("loading page");
-    DataPage page = pageReader.readPage();
-    page.accept(new DataPage.Visitor<Void>() {
-      @Override
-      public Void visit(DataPageV1 dataPageV1) {
-        readPageV1(dataPageV1);
-        return null;
-      }
-      @Override
-      public Void visit(DataPageV2 dataPageV2) {
-        readPageV2(dataPageV2);
-        return null;
-      }
-    });
-  }
-
-  private void initDataReader(Encoding dataEncoding, ByteBufferInputStream in, int valueCount) {
-    ValuesReader previousReader = this.dataColumn;
-
-    this.currentEncoding = dataEncoding;
-    this.pageValueCount = valueCount;
-    this.endOfPageValueCount = readValues + pageValueCount;
-
-    if (dataEncoding.usesDictionary()) {
-      if (dictionary == null) {
-        throw new ParquetDecodingException(
-            "could not read page in col " + path + " as the dictionary was missing for encoding " + dataEncoding);
-      }
-      this.dataColumn = dataEncoding.getDictionaryBasedValuesReader(path, VALUES, dictionary);
-    } else {
-      this.dataColumn = dataEncoding.getValuesReader(path, VALUES);
-    }
-
-    if (dataEncoding.usesDictionary() && converter.hasDictionarySupport()) {
-      bindToDictionary(dictionary);
-    } else {
-      bind(path.getType());
-    }
-
-    try {
-      dataColumn.initFromPage(pageValueCount, in);
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read page in col " + path, e);
-    }
-
-    if (CorruptDeltaByteArrays.requiresSequentialReads(writerVersion, dataEncoding) &&
-        previousReader != null && previousReader instanceof RequiresPreviousReader) {
-      // previous reader can only be set if reading sequentially
-      ((RequiresPreviousReader) dataColumn).setPreviousReader(previousReader);
-    }
-  }
-
-  private void readPageV1(DataPageV1 page) {
-    ValuesReader rlReader = page.getRlEncoding().getValuesReader(path, REPETITION_LEVEL);
-    ValuesReader dlReader = page.getDlEncoding().getValuesReader(path, DEFINITION_LEVEL);
-    this.repetitionLevelColumn = new ValuesReaderIntIterator(rlReader);
-    this.definitionLevelColumn = new ValuesReaderIntIterator(dlReader);
-    try {
-      BytesInput bytes = page.getBytes();
-      LOG.debug("page size {} bytes and {} records", bytes.size(), pageValueCount);
-      LOG.debug("reading repetition levels at 0");
-      ByteBufferInputStream in = bytes.toInputStream();
-      rlReader.initFromPage(pageValueCount, in);
-      LOG.debug("reading definition levels at {}", in.position());
-      dlReader.initFromPage(pageValueCount, in);
-      LOG.debug("reading data at {}", in.position());
-      initDataReader(page.getValueEncoding(), in, page.getValueCount());
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read page " + page + " in col " + path, e);
-    }
-  }
-
-  private void readPageV2(DataPageV2 page) {
-    this.repetitionLevelColumn = newRLEIterator(path.getMaxRepetitionLevel(), page.getRepetitionLevels());
-    this.definitionLevelColumn = newRLEIterator(path.getMaxDefinitionLevel(), page.getDefinitionLevels());
-    LOG.debug("page data size {} bytes and {} records", page.getData().size(), pageValueCount);
-    try {
-      initDataReader(page.getDataEncoding(), page.getData().toInputStream(), page.getValueCount());
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read page " + page + " in col " + path, e);
-    }
-  }
-
-  private IntIterator newRLEIterator(int maxLevel, BytesInput bytes) {
-    try {
-      if (maxLevel == 0) {
-        return new NullIntIterator();
-      }
-      return new RLEIntIterator(
-          new RunLengthBitPackingHybridDecoder(
-              BytesUtils.getWidthFromMaxInt(maxLevel),
-              bytes.toInputStream()));
-    } catch (IOException e) {
-      throw new ParquetDecodingException("could not read levels in page for col " + path, e);
-    }
-  }
-
-  private boolean isPageFullyConsumed() {
-    return readValues >= endOfPageValueCount;
-  }
-
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#consume()
-   */
   @Override
-  public void consume() {
-    checkRead();
-    valueRead = false;
+  boolean skipRL(int rl) {
+    return false;
   }
 
-  /**
-   * {@inheritDoc}
-   * @see org.apache.parquet.column.ColumnReader#getTotalValueCount()
-   */
   @Override
-  public long getTotalValueCount() {
-    return totalValueCount;
-  }
-
-  static abstract class IntIterator {
-    abstract int nextInt();
-  }
-
-  static class ValuesReaderIntIterator extends IntIterator {
-    ValuesReader delegate;
-
-    public ValuesReaderIntIterator(ValuesReader delegate) {
-      super();
-      this.delegate = delegate;
-    }
-
-    @Override
-    int nextInt() {
-      return delegate.readInteger();
-    }
-  }
-
-  static class RLEIntIterator extends IntIterator {
-    RunLengthBitPackingHybridDecoder delegate;
-
-    public RLEIntIterator(RunLengthBitPackingHybridDecoder delegate) {
-      this.delegate = delegate;
-    }
-
-    @Override
-    int nextInt() {
-      try {
-        return delegate.readInt();
-      } catch (IOException e) {
-        throw new ParquetDecodingException(e);
-      }
-    }
-  }
-
-  private static final class NullIntIterator extends IntIterator {
-    @Override
-    int nextInt() {
-      return 0;
-    }
+  void newPageInitialized(DataPage page) {
   }
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
similarity index 59%
copy from parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
copy to parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
index 7574ced..5cd7d87 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreBase.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -23,7 +23,6 @@ import static java.lang.Math.min;
 import static java.util.Collections.unmodifiableMap;
 
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
@@ -37,37 +36,79 @@ import org.apache.parquet.column.page.PageWriteStore;
 import org.apache.parquet.column.page.PageWriter;
 import org.apache.parquet.schema.MessageType;
 
-public class ColumnWriteStoreV2 implements ColumnWriteStore {
+/**
+ * Base implementation for {@link ColumnWriteStore} to be extended to specialize for V1 and V2 pages.
+ */
+abstract class ColumnWriteStoreBase implements ColumnWriteStore {
+
+  // Used to support the deprecated workflow of ColumnWriteStoreV1 (lazy init of ColumnWriters)
+  private interface ColumnWriterProvider {
+    ColumnWriter getColumnWriter(ColumnDescriptor path);
+  }
+
+  private final ColumnWriterProvider columnWriterProvider;
 
   // will flush even if size bellow the threshold by this much to facilitate page alignment
   private static final float THRESHOLD_TOLERANCE_RATIO = 0.1f; // 10 %
 
-  private final Map<ColumnDescriptor, ColumnWriterV2> columns;
-  private final Collection<ColumnWriterV2> writers;
+  private final Map<ColumnDescriptor, ColumnWriterBase> columns;
   private final ParquetProperties props;
   private final long thresholdTolerance;
   private long rowCount;
   private long rowCountForNextSizeCheck;
 
-  public ColumnWriteStoreV2(
+  // To be used by the deprecated constructor of ColumnWriteStoreV1
+  @Deprecated
+  ColumnWriteStoreBase(
+      final PageWriteStore pageWriteStore,
+      final ParquetProperties props) {
+    this.props = props;
+    this.thresholdTolerance = (long) (props.getPageSizeThreshold() * THRESHOLD_TOLERANCE_RATIO);
+
+    this.columns = new TreeMap<>();
+
+    this.rowCountForNextSizeCheck = props.getMinRowCountForPageSizeCheck();
+
+    columnWriterProvider = new ColumnWriterProvider() {
+      @Override
+      public ColumnWriter getColumnWriter(ColumnDescriptor path) {
+        ColumnWriterBase column = columns.get(path);
+        if (column == null) {
+          column = createColumnWriter(path, pageWriteStore.getPageWriter(path), props);
+          columns.put(path, column);
+        }
+        return column;
+      }
+    };
+  }
+
+  ColumnWriteStoreBase(
       MessageType schema,
       PageWriteStore pageWriteStore,
       ParquetProperties props) {
     this.props = props;
-    this.thresholdTolerance = (long)(props.getPageSizeThreshold() * THRESHOLD_TOLERANCE_RATIO);
-    Map<ColumnDescriptor, ColumnWriterV2> mcolumns = new TreeMap<ColumnDescriptor, ColumnWriterV2>();
+    this.thresholdTolerance = (long) (props.getPageSizeThreshold() * THRESHOLD_TOLERANCE_RATIO);
+    Map<ColumnDescriptor, ColumnWriterBase> mcolumns = new TreeMap<>();
     for (ColumnDescriptor path : schema.getColumns()) {
       PageWriter pageWriter = pageWriteStore.getPageWriter(path);
-      mcolumns.put(path, new ColumnWriterV2(path, pageWriter, props));
+      mcolumns.put(path, createColumnWriter(path, pageWriter, props));
     }
     this.columns = unmodifiableMap(mcolumns);
-    this.writers = this.columns.values();
 
     this.rowCountForNextSizeCheck = props.getMinRowCountForPageSizeCheck();
+
+    columnWriterProvider = new ColumnWriterProvider() {
+      @Override
+      public ColumnWriter getColumnWriter(ColumnDescriptor path) {
+        return columns.get(path);
+      }
+    };
   }
 
+  abstract ColumnWriterBase createColumnWriter(ColumnDescriptor path, PageWriter pageWriter, ParquetProperties props);
+
   public ColumnWriter getColumnWriter(ColumnDescriptor path) {
-    return columns.get(path);
+    return columnWriterProvider.getColumnWriter(path);
   }
 
   public Set<ColumnDescriptor> getColumnDescriptors() {
@@ -76,19 +117,19 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
 
   @Override
   public String toString() {
-      StringBuilder sb = new StringBuilder();
-      for (Entry<ColumnDescriptor, ColumnWriterV2> entry : columns.entrySet()) {
-        sb.append(Arrays.toString(entry.getKey().getPath())).append(": ");
-        sb.append(entry.getValue().getTotalBufferedSize()).append(" bytes");
-        sb.append("\n");
-      }
-      return sb.toString();
+    StringBuilder sb = new StringBuilder();
+    for (Entry<ColumnDescriptor, ColumnWriterBase> entry : columns.entrySet()) {
+      sb.append(Arrays.toString(entry.getKey().getPath())).append(": ");
+      sb.append(entry.getValue().getTotalBufferedSize()).append(" bytes");
+      sb.append("\n");
+    }
+    return sb.toString();
   }
 
   @Override
   public long getAllocatedSize() {
     long total = 0;
-    for (ColumnWriterV2 memColumn : columns.values()) {
+    for (ColumnWriterBase memColumn : columns.values()) {
       total += memColumn.allocatedSize();
     }
     return total;
@@ -97,7 +138,7 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
   @Override
   public long getBufferedSize() {
     long total = 0;
-    for (ColumnWriterV2 memColumn : columns.values()) {
+    for (ColumnWriterBase memColumn : columns.values()) {
       total += memColumn.getTotalBufferedSize();
     }
     return total;
@@ -105,10 +146,10 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
 
   @Override
   public void flush() {
-    for (ColumnWriterV2 memColumn : columns.values()) {
+    for (ColumnWriterBase memColumn : columns.values()) {
       long rows = rowCount - memColumn.getRowsWrittenSoFar();
       if (rows > 0) {
-        memColumn.writePage(rowCount);
+        memColumn.writePage();
       }
       memColumn.finalizeColumnChunk();
     }
@@ -116,24 +157,32 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
 
   public String memUsageString() {
     StringBuilder b = new StringBuilder("Store {\n");
-    for (ColumnWriterV2 memColumn : columns.values()) {
+    for (ColumnWriterBase memColumn : columns.values()) {
       b.append(memColumn.memUsageString(" "));
     }
     b.append("}\n");
     return b.toString();
   }
 
+  public long maxColMemSize() {
+    long max = 0;
+    for (ColumnWriterBase memColumn : columns.values()) {
+      max = Math.max(max, memColumn.getBufferedSizeInMemory());
+    }
+    return max;
+  }
+
   @Override
   public void close() {
     flush(); // calling flush() here to keep it consistent with the behavior before merging with master
-    for (ColumnWriterV2 memColumn : columns.values()) {
+    for (ColumnWriterBase memColumn : columns.values()) {
       memColumn.close();
     }
   }
 
   @Override
   public void endRecord() {
-    ++ rowCount;
+    ++rowCount;
     if (rowCount >= rowCountForNextSizeCheck) {
       sizeCheck();
     }
@@ -141,18 +190,18 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
 
   private void sizeCheck() {
     long minRecordToWait = Long.MAX_VALUE;
-    for (ColumnWriterV2 writer : writers) {
+    for (ColumnWriterBase writer : columns.values()) {
       long usedMem = writer.getCurrentPageBufferedSize();
       long rows = rowCount - writer.getRowsWrittenSoFar();
       long remainingMem = props.getPageSizeThreshold() - usedMem;
       if (remainingMem <= thresholdTolerance) {
-        writer.writePage(rowCount);
+        writer.writePage();
         remainingMem = props.getPageSizeThreshold();
       }
       long rowsToFillPage =
           usedMem == 0 ?
               props.getMaxRowCountForPageSizeCheck()
-              : (long)((float)rows) / usedMem * remainingMem;
+              : (long) ((float) rows) / usedMem * remainingMem;
       if (rowsToFillPage < minRecordToWait) {
         minRecordToWait = rowsToFillPage;
       }
@@ -161,7 +210,7 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
       minRecordToWait = props.getMinRowCountForPageSizeCheck();
     }
 
-    if(props.estimateNextSizeCheck()) {
+    if (props.estimateNextSizeCheck()) {
       // will check again halfway if between min and max
       rowCountForNextSizeCheck = rowCount +
           min(
@@ -171,5 +220,4 @@ public class ColumnWriteStoreV2 implements ColumnWriteStore {
       rowCountForNextSizeCheck = rowCount + props.getMinRowCountForPageSizeCheck();
     }
   }
-
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
index 93a497f..7258423 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV1.java
@@ -18,121 +18,26 @@
  */
 package org.apache.parquet.column.impl;
 
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-
-import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.ColumnWriteStore;
-import org.apache.parquet.column.ColumnWriter;
 import org.apache.parquet.column.ParquetProperties;
-import org.apache.parquet.column.ParquetProperties.WriterVersion;
 import org.apache.parquet.column.page.PageWriteStore;
 import org.apache.parquet.column.page.PageWriter;
+import org.apache.parquet.schema.MessageType;
 
-public class ColumnWriteStoreV1 implements ColumnWriteStore {
-
-  private final Map<ColumnDescriptor, ColumnWriterV1> columns = new TreeMap<ColumnDescriptor, ColumnWriterV1>();
-  private final PageWriteStore pageWriteStore;
-  private final ParquetProperties props;
-
-  public ColumnWriteStoreV1(PageWriteStore pageWriteStore,
-                            ParquetProperties props) {
-    this.pageWriteStore = pageWriteStore;
-    this.props = props;
-  }
-
-  public ColumnWriter getColumnWriter(ColumnDescriptor path) {
-    ColumnWriterV1 column = columns.get(path);
-    if (column == null) {
-      column = newMemColumn(path);
-      columns.put(path, column);
-    }
-    return column;
-  }
-
-  public Set<ColumnDescriptor> getColumnDescriptors() {
-    return columns.keySet();
-  }
-
-  private ColumnWriterV1 newMemColumn(ColumnDescriptor path) {
-    PageWriter pageWriter = pageWriteStore.getPageWriter(path);
-    return new ColumnWriterV1(path, pageWriter, props);
-  }
-
-  @Override
-  public String toString() {
-      StringBuilder sb = new StringBuilder();
-      for (Entry<ColumnDescriptor, ColumnWriterV1> entry : columns.entrySet()) {
-        sb.append(Arrays.toString(entry.getKey().getPath())).append(": ");
-        sb.append(entry.getValue().getBufferedSizeInMemory()).append(" bytes");
-        sb.append("\n");
-      }
-      return sb.toString();
-  }
-
-  @Override
-  public long getAllocatedSize() {
-    Collection<ColumnWriterV1> values = columns.values();
-    long total = 0;
-    for (ColumnWriterV1 memColumn : values) {
-      total += memColumn.allocatedSize();
-    }
-    return total;
-  }
-
-  @Override
-  public long getBufferedSize() {
-    Collection<ColumnWriterV1> values = columns.values();
-    long total = 0;
-    for (ColumnWriterV1 memColumn : values) {
-      total += memColumn.getBufferedSizeInMemory();
-    }
-    return total;
-  }
-
-  @Override
-  public String memUsageString() {
-    StringBuilder b = new StringBuilder("Store {\n");
-    Collection<ColumnWriterV1> values = columns.values();
-    for (ColumnWriterV1 memColumn : values) {
-      b.append(memColumn.memUsageString(" "));
-    }
-    b.append("}\n");
-    return b.toString();
-  }
+public class ColumnWriteStoreV1 extends ColumnWriteStoreBase {
 
-  public long maxColMemSize() {
-    Collection<ColumnWriterV1> values = columns.values();
-    long max = 0;
-    for (ColumnWriterV1 memColumn : values) {
-      max = Math.max(max, memColumn.getBufferedSizeInMemory());
-    }
-    return max;
+  public ColumnWriteStoreV1(MessageType schema, PageWriteStore pageWriteStore, ParquetProperties props) {
+    super(schema, pageWriteStore, props);
   }
 
-  @Override
-  public void flush() {
-    Collection<ColumnWriterV1> values = columns.values();
-    for (ColumnWriterV1 memColumn : values) {
-      memColumn.flush();
-    }
+  @Deprecated
+  public ColumnWriteStoreV1(final PageWriteStore pageWriteStore,
+      final ParquetProperties props) {
+    super(pageWriteStore, props);
   }
 
   @Override
-  public void endRecord() {
-    // V1 does not take record boundaries into account
-  }
-
-  public void close() {
-    Collection<ColumnWriterV1> values = columns.values();
-    for (ColumnWriterV1 memColumn : values) {
-      memColumn.close();
-    }
+  ColumnWriterBase createColumnWriter(ColumnDescriptor path, PageWriter pageWriter, ParquetProperties props) {
+    return new ColumnWriterV1(path, pageWriter, props);
   }
-
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
index 7574ced..bf1090d 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriteStoreV2.java
@@ -18,158 +18,20 @@
  */
 package org.apache.parquet.column.impl;
 
-import static java.lang.Math.max;
-import static java.lang.Math.min;
-import static java.util.Collections.unmodifiableMap;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeMap;
-
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.ColumnWriteStore;
-import org.apache.parquet.column.ColumnWriter;
 import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.column.page.PageWriteStore;
 import org.apache.parquet.column.page.PageWriter;
 import org.apache.parquet.schema.MessageType;
 
-public class ColumnWriteStoreV2 implements ColumnWriteStore {
-
-  // will flush even if size bellow the threshold by this much to facilitate page alignment
-  private static final float THRESHOLD_TOLERANCE_RATIO = 0.1f; // 10 %
-
-  private final Map<ColumnDescriptor, ColumnWriterV2> columns;
-  private final Collection<ColumnWriterV2> writers;
-  private final ParquetProperties props;
-  private final long thresholdTolerance;
-  private long rowCount;
-  private long rowCountForNextSizeCheck;
-
-  public ColumnWriteStoreV2(
-      MessageType schema,
-      PageWriteStore pageWriteStore,
-      ParquetProperties props) {
-    this.props = props;
-    this.thresholdTolerance = (long)(props.getPageSizeThreshold() * THRESHOLD_TOLERANCE_RATIO);
-    Map<ColumnDescriptor, ColumnWriterV2> mcolumns = new TreeMap<ColumnDescriptor, ColumnWriterV2>();
-    for (ColumnDescriptor path : schema.getColumns()) {
-      PageWriter pageWriter = pageWriteStore.getPageWriter(path);
-      mcolumns.put(path, new ColumnWriterV2(path, pageWriter, props));
-    }
-    this.columns = unmodifiableMap(mcolumns);
-    this.writers = this.columns.values();
-
-    this.rowCountForNextSizeCheck = props.getMinRowCountForPageSizeCheck();
-  }
-
-  public ColumnWriter getColumnWriter(ColumnDescriptor path) {
-    return columns.get(path);
-  }
-
-  public Set<ColumnDescriptor> getColumnDescriptors() {
-    return columns.keySet();
-  }
-
-  @Override
-  public String toString() {
-      StringBuilder sb = new StringBuilder();
-      for (Entry<ColumnDescriptor, ColumnWriterV2> entry : columns.entrySet()) {
-        sb.append(Arrays.toString(entry.getKey().getPath())).append(": ");
-        sb.append(entry.getValue().getTotalBufferedSize()).append(" bytes");
-        sb.append("\n");
-      }
-      return sb.toString();
-  }
-
-  @Override
-  public long getAllocatedSize() {
-    long total = 0;
-    for (ColumnWriterV2 memColumn : columns.values()) {
-      total += memColumn.allocatedSize();
-    }
-    return total;
-  }
-
-  @Override
-  public long getBufferedSize() {
-    long total = 0;
-    for (ColumnWriterV2 memColumn : columns.values()) {
-      total += memColumn.getTotalBufferedSize();
-    }
-    return total;
-  }
-
-  @Override
-  public void flush() {
-    for (ColumnWriterV2 memColumn : columns.values()) {
-      long rows = rowCount - memColumn.getRowsWrittenSoFar();
-      if (rows > 0) {
-        memColumn.writePage(rowCount);
-      }
-      memColumn.finalizeColumnChunk();
-    }
-  }
+public class ColumnWriteStoreV2 extends ColumnWriteStoreBase {
 
-  public String memUsageString() {
-    StringBuilder b = new StringBuilder("Store {\n");
-    for (ColumnWriterV2 memColumn : columns.values()) {
-      b.append(memColumn.memUsageString(" "));
-    }
-    b.append("}\n");
-    return b.toString();
+  public ColumnWriteStoreV2(MessageType schema, PageWriteStore pageWriteStore, ParquetProperties props) {
+    super(schema, pageWriteStore, props);
   }
 
   @Override
-  public void close() {
-    flush(); // calling flush() here to keep it consistent with the behavior before merging with master
-    for (ColumnWriterV2 memColumn : columns.values()) {
-      memColumn.close();
-    }
+  ColumnWriterBase createColumnWriter(ColumnDescriptor path, PageWriter pageWriter, ParquetProperties props) {
+    return new ColumnWriterV2(path, pageWriter, props);
   }
-
-  @Override
-  public void endRecord() {
-    ++ rowCount;
-    if (rowCount >= rowCountForNextSizeCheck) {
-      sizeCheck();
-    }
-  }
-
-  private void sizeCheck() {
-    long minRecordToWait = Long.MAX_VALUE;
-    for (ColumnWriterV2 writer : writers) {
-      long usedMem = writer.getCurrentPageBufferedSize();
-      long rows = rowCount - writer.getRowsWrittenSoFar();
-      long remainingMem = props.getPageSizeThreshold() - usedMem;
-      if (remainingMem <= thresholdTolerance) {
-        writer.writePage(rowCount);
-        remainingMem = props.getPageSizeThreshold();
-      }
-      long rowsToFillPage =
-          usedMem == 0 ?
-              props.getMaxRowCountForPageSizeCheck()
-              : (long)((float)rows) / usedMem * remainingMem;
-      if (rowsToFillPage < minRecordToWait) {
-        minRecordToWait = rowsToFillPage;
-      }
-    }
-    if (minRecordToWait == Long.MAX_VALUE) {
-      minRecordToWait = props.getMinRowCountForPageSizeCheck();
-    }
-
-    if(props.estimateNextSizeCheck()) {
-      // will check again halfway if between min and max
-      rowCountForNextSizeCheck = rowCount +
-          min(
-              max(minRecordToWait / 2, props.getMinRowCountForPageSizeCheck()),
-              props.getMaxRowCountForPageSizeCheck());
-    } else {
-      rowCountForNextSizeCheck = rowCount + props.getMinRowCountForPageSizeCheck();
-    }
-  }
-
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
similarity index 66%
copy from parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
copy to parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
index 9abdee8..3788c82 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterBase.java
@@ -1,4 +1,4 @@
-/* 
+/*
  * 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
@@ -6,9 +6,9 @@
  * 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
@@ -20,43 +20,40 @@ package org.apache.parquet.column.impl;
 
 import java.io.IOException;
 
-import org.apache.parquet.Ints;
-import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ColumnWriter;
-import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.column.page.DictionaryPage;
 import org.apache.parquet.column.page.PageWriter;
 import org.apache.parquet.column.statistics.Statistics;
 import org.apache.parquet.column.values.ValuesWriter;
-import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder;
 import org.apache.parquet.io.ParquetEncodingException;
 import org.apache.parquet.io.api.Binary;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
 /**
- * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer.
+ * Base implementation for {@link ColumnWriter} to be extended to specialize for V1 and V2 pages.
  */
-final class ColumnWriterV2 implements ColumnWriter {
-  private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterV2.class);
+abstract class ColumnWriterBase implements ColumnWriter {
+  private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterBase.class);
 
   // By default: Debugging disabled this way (using the "if (DEBUG)" IN the methods) to allow
   // the java compiler (not the JIT) to remove the unused statements during build time.
   private static final boolean DEBUG = false;
 
-  private final ColumnDescriptor path;
-  private final PageWriter pageWriter;
-  private RunLengthBitPackingHybridEncoder repetitionLevelColumn;
-  private RunLengthBitPackingHybridEncoder definitionLevelColumn;
+  final ColumnDescriptor path;
+  final PageWriter pageWriter;
+  private ValuesWriter repetitionLevelColumn;
+  private ValuesWriter definitionLevelColumn;
   private ValuesWriter dataColumn;
   private int valueCount;
 
   private Statistics<?> statistics;
   private long rowsWrittenSoFar = 0;
+  private int pageRowCount;
 
-  public ColumnWriterV2(
+  ColumnWriterBase(
       ColumnDescriptor path,
       PageWriter pageWriter,
       ParquetProperties props) {
@@ -64,11 +61,15 @@ final class ColumnWriterV2 implements ColumnWriter {
     this.pageWriter = pageWriter;
     resetStatistics();
 
-    this.repetitionLevelColumn = props.newRepetitionLevelEncoder(path);
-    this.definitionLevelColumn = props.newDefinitionLevelEncoder(path);
+    this.repetitionLevelColumn = createRLWriter(props, path);
+    this.definitionLevelColumn = createDLWriter(props, path);
     this.dataColumn = props.newValuesWriter(path);
   }
 
+  abstract ValuesWriter createRLWriter(ParquetProperties props, ColumnDescriptor path);
+
+  abstract ValuesWriter createDLWriter(ParquetProperties props, ColumnDescriptor path);
+
   private void log(Object value, int r, int d) {
     LOG.debug("{} {} r:{} d:{}", path, value, r, d);
   }
@@ -78,32 +79,31 @@ final class ColumnWriterV2 implements ColumnWriter {
   }
 
   private void definitionLevel(int definitionLevel) {
-    try {
-      definitionLevelColumn.writeInt(definitionLevel);
-    } catch (IOException e) {
-      throw new ParquetEncodingException("illegal definition level " + definitionLevel + " for column " + path, e);
-    }
+    definitionLevelColumn.writeInteger(definitionLevel);
   }
 
   private void repetitionLevel(int repetitionLevel) {
-    try {
-      repetitionLevelColumn.writeInt(repetitionLevel);
-    } catch (IOException e) {
-      throw new ParquetEncodingException("illegal repetition level " + repetitionLevel + " for column " + path, e);
+    repetitionLevelColumn.writeInteger(repetitionLevel);
+    assert pageRowCount == 0 ? repetitionLevel == 0 : true : "Every page shall start on record boundaries";
+    if (repetitionLevel == 0) {
+      ++pageRowCount;
     }
   }
 
   /**
-   * writes the current null value
+   * Writes the current null value
+   *
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void writeNull(int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(null, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(null, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     statistics.incrementNumNulls();
-    ++ valueCount;
+    ++valueCount;
   }
 
   @Override
@@ -117,109 +117,128 @@ final class ColumnWriterV2 implements ColumnWriter {
   @Override
   public long getBufferedSizeInMemory() {
     return repetitionLevelColumn.getBufferedSize()
-      + definitionLevelColumn.getBufferedSize()
-      + dataColumn.getBufferedSize()
-      + pageWriter.getMemSize();
+        + definitionLevelColumn.getBufferedSize()
+        + dataColumn.getBufferedSize()
+        + pageWriter.getMemSize();
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(double value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeDouble(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(float value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeFloat(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(Binary value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeBytes(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(boolean value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeBoolean(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(int value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeInteger(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
-   * writes the current value
+   * Writes the current value
+   *
    * @param value
    * @param repetitionLevel
    * @param definitionLevel
    */
+  @Override
   public void write(long value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
+    if (DEBUG)
+      log(value, repetitionLevel, definitionLevel);
     repetitionLevel(repetitionLevel);
     definitionLevel(definitionLevel);
     dataColumn.writeLong(value);
     statistics.updateStats(value);
-    ++ valueCount;
+    ++valueCount;
   }
 
   /**
    * Finalizes the Column chunk. Possibly adding extra pages if needed (dictionary, ...)
    * Is called right after writePage
    */
-  public void finalizeColumnChunk() {
+  void finalizeColumnChunk() {
     final DictionaryPage dictionaryPage = dataColumn.toDictPageAndClose();
     if (dictionaryPage != null) {
-      if (DEBUG) LOG.debug("write dictionary");
+      if (DEBUG)
+        LOG.debug("write dictionary");
       try {
         pageWriter.writeDictionaryPage(dictionaryPage);
       } catch (IOException e) {
@@ -230,20 +249,22 @@ final class ColumnWriterV2 implements ColumnWriter {
   }
 
   /**
-   * used to decide when to write a page
+   * Used to decide when to write a page
+   *
    * @return the number of bytes of memory used to buffer the current data
    */
-  public long getCurrentPageBufferedSize() {
+  long getCurrentPageBufferedSize() {
     return repetitionLevelColumn.getBufferedSize()
         + definitionLevelColumn.getBufferedSize()
         + dataColumn.getBufferedSize();
   }
 
   /**
-   * used to decide when to write a page or row group
+   * Used to decide when to write a page or row group
+   *
    * @return the number of bytes of memory used to buffer the current data and the previously written pages
    */
-  public long getTotalBufferedSize() {
+  long getTotalBufferedSize() {
     return repetitionLevelColumn.getBufferedSize()
         + definitionLevelColumn.getBufferedSize()
         + dataColumn.getBufferedSize()
@@ -253,18 +274,19 @@ final class ColumnWriterV2 implements ColumnWriter {
   /**
    * @return actual memory used
    */
-  public long allocatedSize() {
+  long allocatedSize() {
     return repetitionLevelColumn.getAllocatedSize()
-    + definitionLevelColumn.getAllocatedSize()
-    + dataColumn.getAllocatedSize()
-    + pageWriter.allocatedSize();
+        + definitionLevelColumn.getAllocatedSize()
+        + dataColumn.getAllocatedSize()
+        + pageWriter.allocatedSize();
   }
 
   /**
-   * @param indent a prefix to format lines
+   * @param indent
+   *          a prefix to format lines
    * @return a formatted string showing how memory is used
    */
-  public String memUsageString(String indent) {
+  String memUsageString(String indent) {
     StringBuilder b = new StringBuilder(indent).append(path).append(" {\n");
     b.append(indent).append(" r:").append(repetitionLevelColumn.getAllocatedSize()).append(" bytes\n");
     b.append(indent).append(" d:").append(definitionLevelColumn.getAllocatedSize()).append(" bytes\n");
@@ -275,32 +297,19 @@ final class ColumnWriterV2 implements ColumnWriter {
     return b.toString();
   }
 
-  public long getRowsWrittenSoFar() {
+  long getRowsWrittenSoFar() {
     return this.rowsWrittenSoFar;
   }
 
   /**
-   * writes the current data to a new page in the page store
-   * @param rowCount how many rows have been written so far
+   * Writes the current data to a new page in the page store
    */
-  public void writePage(long rowCount) {
-    int pageRowCount = Ints.checkedCast(rowCount - rowsWrittenSoFar);
-    this.rowsWrittenSoFar = rowCount;
-    if (DEBUG) LOG.debug("write page");
+  void writePage() {
+    this.rowsWrittenSoFar += pageRowCount;
+    if (DEBUG)
+      LOG.debug("write page");
     try {
-      // TODO: rework this API. Those must be called *in that order*
-      BytesInput bytes = dataColumn.getBytes();
-      Encoding encoding = dataColumn.getEncoding();
-      pageWriter.writePageV2(
-          pageRowCount,
-          Ints.checkedCast(statistics.getNumNulls()),
-          valueCount,
-          path.getMaxRepetitionLevel() == 0 ? BytesInput.empty() : repetitionLevelColumn.toBytes(),
-          path.getMaxDefinitionLevel() == 0 ? BytesInput.empty() : definitionLevelColumn.toBytes(),
-          encoding,
-          bytes,
-          statistics
-          );
+      writePage(pageRowCount, valueCount, statistics, repetitionLevelColumn, definitionLevelColumn, dataColumn);
     } catch (IOException e) {
       throw new ParquetEncodingException("could not write page for " + path, e);
     }
@@ -309,5 +318,9 @@ final class ColumnWriterV2 implements ColumnWriter {
     dataColumn.reset();
     valueCount = 0;
     resetStatistics();
+    pageRowCount = 0;
   }
+
+  abstract void writePage(int rowCount, int valueCount, Statistics<?> statistics, ValuesWriter repetitionLevels,
+      ValuesWriter definitionLevels, ValuesWriter values) throws IOException;
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
index c1f5d67..646e31a 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV1.java
@@ -23,261 +23,40 @@ import static org.apache.parquet.bytes.BytesInput.concat;
 import java.io.IOException;
 
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.ColumnWriter;
 import org.apache.parquet.column.ParquetProperties;
-import org.apache.parquet.column.page.DictionaryPage;
 import org.apache.parquet.column.page.PageWriter;
 import org.apache.parquet.column.statistics.Statistics;
 import org.apache.parquet.column.values.ValuesWriter;
-import org.apache.parquet.io.ParquetEncodingException;
-import org.apache.parquet.io.api.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer.
  */
-final class ColumnWriterV1 implements ColumnWriter {
-  private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterV1.class);
+final class ColumnWriterV1 extends ColumnWriterBase {
 
-  // By default: Debugging disabled this way (using the "if (DEBUG)" IN the methods) to allow
-  // the java compiler (not the JIT) to remove the unused statements during build time.
-  private static final boolean DEBUG = false;
-
-  private final ColumnDescriptor path;
-  private final PageWriter pageWriter;
-  private final ParquetProperties props;
-
-  private ValuesWriter repetitionLevelColumn;
-  private ValuesWriter definitionLevelColumn;
-  private ValuesWriter dataColumn;
-  private int valueCount;
-  private int valueCountForNextSizeCheck;
-
-  private Statistics statistics;
-
-  public ColumnWriterV1(ColumnDescriptor path, PageWriter pageWriter,
-                        ParquetProperties props) {
-    this.path = path;
-    this.pageWriter = pageWriter;
-    this.props = props;
-
-    // initial check of memory usage. So that we have enough data to make an initial prediction
-    this.valueCountForNextSizeCheck = props.getMinRowCountForPageSizeCheck();
-
-    resetStatistics();
-
-    this.repetitionLevelColumn = props.newRepetitionLevelWriter(path);
-    this.definitionLevelColumn = props.newDefinitionLevelWriter(path);
-    this.dataColumn = props.newValuesWriter(path);
-  }
-
-  private void log(Object value, int r, int d) {
-    if (DEBUG) LOG.debug( "{} {} r:{} d:{}", path, value, r, d);
-  }
-
-  private void resetStatistics() {
-    this.statistics = Statistics.createStats(this.path.getPrimitiveType());
-  }
-
-  /**
-   * Counts how many values have been written and checks the memory usage to flush the page when we reach the page threshold.
-   *
-   * We measure the memory used when we reach the mid point toward our estimated count.
-   * We then update the estimate and flush the page if we reached the threshold.
-   *
-   * That way we check the memory size log2(n) times.
-   *
-   */
-  private void accountForValueWritten() {
-    ++ valueCount;
-    if (valueCount > valueCountForNextSizeCheck) {
-      // not checking the memory used for every value
-      long memSize = repetitionLevelColumn.getBufferedSize()
-          + definitionLevelColumn.getBufferedSize()
-          + dataColumn.getBufferedSize();
-      if (memSize > props.getPageSizeThreshold()) {
-        // we will write the current page and check again the size at the predicted middle of next page
-        if (props.estimateNextSizeCheck()) {
-          valueCountForNextSizeCheck = valueCount / 2;
-        } else {
-          valueCountForNextSizeCheck = props.getMinRowCountForPageSizeCheck();
-        }
-        writePage();
-      } else if (props.estimateNextSizeCheck()) {
-        // not reached the threshold, will check again midway
-        valueCountForNextSizeCheck = (int)(valueCount + ((float)valueCount * props.getPageSizeThreshold() / memSize)) / 2 + 1;
-      } else {
-        valueCountForNextSizeCheck += props.getMinRowCountForPageSizeCheck();
-      }
-    }
-  }
-
-  private void updateStatisticsNumNulls() {
-    statistics.incrementNumNulls();
-  }
-
-  private void updateStatistics(int value) {
-    statistics.updateStats(value);
-  }
-
-  private void updateStatistics(long value) {
-    statistics.updateStats(value);
-  }
-
-  private void updateStatistics(float value) {
-    statistics.updateStats(value);
-  }
-
-  private void updateStatistics(double value) {
-   statistics.updateStats(value);
-  }
-
-  private void updateStatistics(Binary value) {
-   statistics.updateStats(value);
-  }
-
-  private void updateStatistics(boolean value) {
-   statistics.updateStats(value);
-  }
-
-  private void writePage() {
-    if (DEBUG) LOG.debug("write page");
-    try {
-      pageWriter.writePage(
-          concat(repetitionLevelColumn.getBytes(), definitionLevelColumn.getBytes(), dataColumn.getBytes()),
-          valueCount,
-          statistics,
-          repetitionLevelColumn.getEncoding(),
-          definitionLevelColumn.getEncoding(),
-          dataColumn.getEncoding());
-    } catch (IOException e) {
-      throw new ParquetEncodingException("could not write page for " + path, e);
-    }
-    repetitionLevelColumn.reset();
-    definitionLevelColumn.reset();
-    dataColumn.reset();
-    valueCount = 0;
-    resetStatistics();
-  }
-
-  @Override
-  public void writeNull(int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(null, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    updateStatisticsNumNulls();
-    accountForValueWritten();
-  }
-
-  @Override
-  public void write(double value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeDouble(value);
-    updateStatistics(value);
-    accountForValueWritten();
-  }
-
-  @Override
-  public void write(float value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeFloat(value);
-    updateStatistics(value);
-    accountForValueWritten();
-  }
-
-  @Override
-  public void write(Binary value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeBytes(value);
-    updateStatistics(value);
-    accountForValueWritten();
+  ColumnWriterV1(ColumnDescriptor path, PageWriter pageWriter, ParquetProperties props) {
+    super(path, pageWriter, props);
   }
 
   @Override
-  public void write(boolean value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeBoolean(value);
-    updateStatistics(value);
-    accountForValueWritten();
+  ValuesWriter createRLWriter(ParquetProperties props, ColumnDescriptor path) {
+    return props.newRepetitionLevelWriter(path);
   }
 
   @Override
-  public void write(int value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeInteger(value);
-    updateStatistics(value);
-    accountForValueWritten();
+  ValuesWriter createDLWriter(ParquetProperties props, ColumnDescriptor path) {
+    return props.newDefinitionLevelWriter(path);
   }
 
   @Override
-  public void write(long value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevelColumn.writeInteger(repetitionLevel);
-    definitionLevelColumn.writeInteger(definitionLevel);
-    dataColumn.writeLong(value);
-    updateStatistics(value);
-    accountForValueWritten();
-  }
-
-  public void flush() {
-    if (valueCount > 0) {
-      writePage();
-    }
-    final DictionaryPage dictionaryPage = dataColumn.toDictPageAndClose();
-    if (dictionaryPage != null) {
-      if (DEBUG) LOG.debug("write dictionary");
-      try {
-        pageWriter.writeDictionaryPage(dictionaryPage);
-      } catch (IOException e) {
-        throw new ParquetEncodingException("could not write dictionary page for " + path, e);
-      }
-      dataColumn.resetDictionary();
-    }
-  }
-
-  @Override
-  public void close() {
-    flush();
-    // Close the Values writers.
-    repetitionLevelColumn.close();
-    definitionLevelColumn.close();
-    dataColumn.close();
-  }
-
-  @Override
-  public long getBufferedSizeInMemory() {
-    return repetitionLevelColumn.getBufferedSize()
-        + definitionLevelColumn.getBufferedSize()
-        + dataColumn.getBufferedSize()
-        + pageWriter.getMemSize();
-  }
-
-  public long allocatedSize() {
-    return repetitionLevelColumn.getAllocatedSize()
-    + definitionLevelColumn.getAllocatedSize()
-    + dataColumn.getAllocatedSize()
-    + pageWriter.allocatedSize();
-  }
-
-  public String memUsageString(String indent) {
-    StringBuilder b = new StringBuilder(indent).append(path).append(" {\n");
-    b.append(repetitionLevelColumn.memUsageString(indent + "  r:")).append("\n");
-    b.append(definitionLevelColumn.memUsageString(indent + "  d:")).append("\n");
-    b.append(dataColumn.memUsageString(indent + "  data:")).append("\n");
-    b.append(pageWriter.memUsageString(indent + "  pages:")).append("\n");
-    b.append(indent).append(String.format("  total: %,d/%,d", getBufferedSizeInMemory(), allocatedSize())).append("\n");
-    b.append(indent).append("}\n");
-    return b.toString();
+  void writePage(int rowCount, int valueCount, Statistics<?> statistics, ValuesWriter repetitionLevels,
+      ValuesWriter definitionLevels, ValuesWriter values) throws IOException {
+    pageWriter.writePage(
+        concat(repetitionLevels.getBytes(), definitionLevels.getBytes(), values.getBytes()),
+        valueCount,
+        rowCount,
+        statistics,
+        repetitionLevels.getEncoding(),
+        definitionLevels.getEncoding(),
+        values.getEncoding());
   }
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
index 9abdee8..04076c9 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/ColumnWriterV2.java
@@ -23,291 +23,67 @@ import java.io.IOException;
 import org.apache.parquet.Ints;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.ColumnDescriptor;
-import org.apache.parquet.column.ColumnWriter;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.ParquetProperties;
-import org.apache.parquet.column.page.DictionaryPage;
 import org.apache.parquet.column.page.PageWriter;
 import org.apache.parquet.column.statistics.Statistics;
 import org.apache.parquet.column.values.ValuesWriter;
+import org.apache.parquet.column.values.bitpacking.DevNullValuesWriter;
 import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridEncoder;
+import org.apache.parquet.column.values.rle.RunLengthBitPackingHybridValuesWriter;
 import org.apache.parquet.io.ParquetEncodingException;
-import org.apache.parquet.io.api.Binary;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
 /**
  * Writes (repetition level, definition level, value) triplets and deals with writing pages to the underlying layer.
  */
-final class ColumnWriterV2 implements ColumnWriter {
-  private static final Logger LOG = LoggerFactory.getLogger(ColumnWriterV2.class);
+final class ColumnWriterV2 extends ColumnWriterBase {
 
-  // By default: Debugging disabled this way (using the "if (DEBUG)" IN the methods) to allow
-  // the java compiler (not the JIT) to remove the unused statements during build time.
-  private static final boolean DEBUG = false;
-
-  private final ColumnDescriptor path;
-  private final PageWriter pageWriter;
-  private RunLengthBitPackingHybridEncoder repetitionLevelColumn;
-  private RunLengthBitPackingHybridEncoder definitionLevelColumn;
-  private ValuesWriter dataColumn;
-  private int valueCount;
-
-  private Statistics<?> statistics;
-  private long rowsWrittenSoFar = 0;
-
-  public ColumnWriterV2(
-      ColumnDescriptor path,
-      PageWriter pageWriter,
-      ParquetProperties props) {
-    this.path = path;
-    this.pageWriter = pageWriter;
-    resetStatistics();
-
-    this.repetitionLevelColumn = props.newRepetitionLevelEncoder(path);
-    this.definitionLevelColumn = props.newDefinitionLevelEncoder(path);
-    this.dataColumn = props.newValuesWriter(path);
-  }
-
-  private void log(Object value, int r, int d) {
-    LOG.debug("{} {} r:{} d:{}", path, value, r, d);
-  }
-
-  private void resetStatistics() {
-    this.statistics = Statistics.createStats(path.getPrimitiveType());
-  }
-
-  private void definitionLevel(int definitionLevel) {
-    try {
-      definitionLevelColumn.writeInt(definitionLevel);
-    } catch (IOException e) {
-      throw new ParquetEncodingException("illegal definition level " + definitionLevel + " for column " + path, e);
+  // Extending the original implementation to not to write the size of the data as the original writer would
+  private static class RLEWriterForV2 extends RunLengthBitPackingHybridValuesWriter {
+    public RLEWriterForV2(RunLengthBitPackingHybridEncoder encoder) {
+      super(encoder);
     }
-  }
 
-  private void repetitionLevel(int repetitionLevel) {
-    try {
-      repetitionLevelColumn.writeInt(repetitionLevel);
-    } catch (IOException e) {
-      throw new ParquetEncodingException("illegal repetition level " + repetitionLevel + " for column " + path, e);
-    }
-  }
-
-  /**
-   * writes the current null value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void writeNull(int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(null, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    statistics.incrementNumNulls();
-    ++ valueCount;
-  }
-
-  @Override
-  public void close() {
-    // Close the Values writers.
-    repetitionLevelColumn.close();
-    definitionLevelColumn.close();
-    dataColumn.close();
-  }
-
-  @Override
-  public long getBufferedSizeInMemory() {
-    return repetitionLevelColumn.getBufferedSize()
-      + definitionLevelColumn.getBufferedSize()
-      + dataColumn.getBufferedSize()
-      + pageWriter.getMemSize();
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(double value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeDouble(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(float value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeFloat(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(Binary value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeBytes(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(boolean value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeBoolean(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(int value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeInteger(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * writes the current value
-   * @param value
-   * @param repetitionLevel
-   * @param definitionLevel
-   */
-  public void write(long value, int repetitionLevel, int definitionLevel) {
-    if (DEBUG) log(value, repetitionLevel, definitionLevel);
-    repetitionLevel(repetitionLevel);
-    definitionLevel(definitionLevel);
-    dataColumn.writeLong(value);
-    statistics.updateStats(value);
-    ++ valueCount;
-  }
-
-  /**
-   * Finalizes the Column chunk. Possibly adding extra pages if needed (dictionary, ...)
-   * Is called right after writePage
-   */
-  public void finalizeColumnChunk() {
-    final DictionaryPage dictionaryPage = dataColumn.toDictPageAndClose();
-    if (dictionaryPage != null) {
-      if (DEBUG) LOG.debug("write dictionary");
+    @Override
+    public BytesInput getBytes() {
       try {
-        pageWriter.writeDictionaryPage(dictionaryPage);
+        return encoder.toBytes();
       } catch (IOException e) {
-        throw new ParquetEncodingException("could not write dictionary page for " + path, e);
+        throw new ParquetEncodingException(e);
       }
-      dataColumn.resetDictionary();
     }
   }
 
-  /**
-   * used to decide when to write a page
-   * @return the number of bytes of memory used to buffer the current data
-   */
-  public long getCurrentPageBufferedSize() {
-    return repetitionLevelColumn.getBufferedSize()
-        + definitionLevelColumn.getBufferedSize()
-        + dataColumn.getBufferedSize();
-  }
-
-  /**
-   * used to decide when to write a page or row group
-   * @return the number of bytes of memory used to buffer the current data and the previously written pages
-   */
-  public long getTotalBufferedSize() {
-    return repetitionLevelColumn.getBufferedSize()
-        + definitionLevelColumn.getBufferedSize()
-        + dataColumn.getBufferedSize()
-        + pageWriter.getMemSize();
-  }
+  private static final ValuesWriter NULL_WRITER = new DevNullValuesWriter();
 
-  /**
-   * @return actual memory used
-   */
-  public long allocatedSize() {
-    return repetitionLevelColumn.getAllocatedSize()
-    + definitionLevelColumn.getAllocatedSize()
-    + dataColumn.getAllocatedSize()
-    + pageWriter.allocatedSize();
+  ColumnWriterV2(ColumnDescriptor path, PageWriter pageWriter, ParquetProperties props) {
+    super(path, pageWriter, props);
   }
 
-  /**
-   * @param indent a prefix to format lines
-   * @return a formatted string showing how memory is used
-   */
-  public String memUsageString(String indent) {
-    StringBuilder b = new StringBuilder(indent).append(path).append(" {\n");
-    b.append(indent).append(" r:").append(repetitionLevelColumn.getAllocatedSize()).append(" bytes\n");
-    b.append(indent).append(" d:").append(definitionLevelColumn.getAllocatedSize()).append(" bytes\n");
-    b.append(dataColumn.memUsageString(indent + "  data:")).append("\n");
-    b.append(pageWriter.memUsageString(indent + "  pages:")).append("\n");
-    b.append(indent).append(String.format("  total: %,d/%,d", getTotalBufferedSize(), allocatedSize())).append("\n");
-    b.append(indent).append("}\n");
-    return b.toString();
+  @Override
+  ValuesWriter createRLWriter(ParquetProperties props, ColumnDescriptor path) {
+    return path.getMaxRepetitionLevel() == 0 ? NULL_WRITER : new RLEWriterForV2(props.newRepetitionLevelEncoder(path));
   }
 
-  public long getRowsWrittenSoFar() {
-    return this.rowsWrittenSoFar;
+  @Override
+  ValuesWriter createDLWriter(ParquetProperties props, ColumnDescriptor path) {
+    return path.getMaxDefinitionLevel() == 0 ? NULL_WRITER : new RLEWriterForV2(props.newDefinitionLevelEncoder(path));
   }
 
-  /**
-   * writes the current data to a new page in the page store
-   * @param rowCount how many rows have been written so far
-   */
-  public void writePage(long rowCount) {
-    int pageRowCount = Ints.checkedCast(rowCount - rowsWrittenSoFar);
-    this.rowsWrittenSoFar = rowCount;
-    if (DEBUG) LOG.debug("write page");
-    try {
-      // TODO: rework this API. Those must be called *in that order*
-      BytesInput bytes = dataColumn.getBytes();
-      Encoding encoding = dataColumn.getEncoding();
-      pageWriter.writePageV2(
-          pageRowCount,
-          Ints.checkedCast(statistics.getNumNulls()),
-          valueCount,
-          path.getMaxRepetitionLevel() == 0 ? BytesInput.empty() : repetitionLevelColumn.toBytes(),
-          path.getMaxDefinitionLevel() == 0 ? BytesInput.empty() : definitionLevelColumn.toBytes(),
-          encoding,
-          bytes,
-          statistics
-          );
-    } catch (IOException e) {
-      throw new ParquetEncodingException("could not write page for " + path, e);
-    }
-    repetitionLevelColumn.reset();
-    definitionLevelColumn.reset();
-    dataColumn.reset();
-    valueCount = 0;
-    resetStatistics();
+  @Override
+  void writePage(int rowCount, int valueCount, Statistics<?> statistics, ValuesWriter repetitionLevels,
+      ValuesWriter definitionLevels, ValuesWriter values) throws IOException {
+    // TODO: rework this API. The bytes shall be retrieved before the encoding (encoding might be different otherwise)
+    BytesInput bytes = values.getBytes();
+    Encoding encoding = values.getEncoding();
+    pageWriter.writePageV2(
+        rowCount,
+        Ints.checkedCast(statistics.getNumNulls()),
+        valueCount,
+        repetitionLevels.getBytes(),
+        definitionLevels.getBytes(),
+        encoding,
+        bytes,
+        statistics);
   }
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/impl/SynchronizingColumnReader.java b/parquet-column/src/main/java/org/apache/parquet/column/impl/SynchronizingColumnReader.java
new file mode 100644
index 0000000..50f05c8
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/column/impl/SynchronizingColumnReader.java
@@ -0,0 +1,111 @@
+/*
+ * 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.parquet.column.impl;
+
+import java.util.PrimitiveIterator;
+
+import org.apache.parquet.VersionParser.ParsedVersion;
+import org.apache.parquet.column.ColumnDescriptor;
+import org.apache.parquet.column.ColumnReader;
+import org.apache.parquet.column.page.DataPage;
+import org.apache.parquet.column.page.PageReader;
+import org.apache.parquet.io.RecordReader;
+import org.apache.parquet.io.api.PrimitiveConverter;
+
+/**
+ * A {@link ColumnReader} implementation for utilizing indexes. When filtering using column indexes we might skip
+ * reading some pages for different columns. Because the rows are not aligned between the pages of the different columns
+ * it might be required to skip some values in this {@link ColumnReader} so we provide only the required values for the
+ * higher API ({@link RecordReader}) and they do not need to handle or know about the skipped pages. The values (and the
+ * related rl and dl) are skipped based on the iterator of the required row indexes and the first row index of each
+ * page.<br>
+ * For example:
+ *
+ * <pre>
+ * rows   col1   col2   col3
+ *      ┌──────┬──────┬──────┐
+ *   0  │  p0  │      │      │
+ *      ╞══════╡  p0  │  p0  │
+ *  20  │ p1(X)│------│------│
+ *      ╞══════╪══════╡      │
+ *  40  │ p2(X)│      │------│
+ *      ╞══════╡ p1(X)╞══════╡
+ *  60  │ p3(X)│      │------│
+ *      ╞══════╪══════╡      │
+ *  80  │  p4  │      │  p1  │
+ *      ╞══════╡  p2  │      │
+ * 100  │  p5  │      │      │
+ *      └──────┴──────┴──────┘
+ * </pre>
+ *
+ * The pages 1, 2, 3 in col1 are skipped so we have to skip the rows [20, 79]. Because page 1 in col2 contains values
+ * only for the rows [40, 79] we skip this entire page as well. To synchronize the row reading we have to skip the
+ * values (and the related rl and dl) for the rows [20, 39] in the end of the page 0 for col2. Similarly, we have to
+ * skip values while reading page0 and page1 for col3.
+ */
+class SynchronizingColumnReader extends ColumnReaderBase {
+
+  private final PrimitiveIterator.OfLong rowIndexes;
+  private long currentRow;
+  private long targetRow;
+  private long lastRowInPage;
+  private int valuesReadFromPage;
+
+  SynchronizingColumnReader(ColumnDescriptor path, PageReader pageReader, PrimitiveConverter converter,
+      ParsedVersion writerVersion, PrimitiveIterator.OfLong rowIndexes) {
+    super(path, pageReader, converter, writerVersion);
+    this.rowIndexes = rowIndexes;
+    targetRow = Long.MIN_VALUE;
+    consume();
+  }
+
+  @Override
+  boolean isPageFullyConsumed() {
+    return getPageValueCount() <= valuesReadFromPage || lastRowInPage < targetRow;
+  }
+
+  @Override
+  boolean isFullyConsumed() {
+    return !rowIndexes.hasNext();
+  }
+
+  @Override
+  boolean skipRL(int rl) {
+    ++valuesReadFromPage;
+    if (rl == 0) {
+      ++currentRow;
+      if (currentRow > targetRow) {
+        targetRow = rowIndexes.hasNext() ? rowIndexes.nextLong() : Long.MAX_VALUE;
+      }
+    }
+    return currentRow < targetRow;
+  }
+
+  @Override
+  protected void newPageInitialized(DataPage page) {
+    long firstRowIndex = page.getFirstRowIndex()
+        .orElseThrow(() -> new IllegalArgumentException("Missing firstRowIndex for synchronizing values"));
+    int rowCount = page.getIndexRowCount()
+        .orElseThrow(() -> new IllegalArgumentException("Missing rowCount for synchronizing values"));
+    currentRow = firstRowIndex - 1;
+    lastRowInPage = firstRowIndex + rowCount - 1;
+    valuesReadFromPage = 0;
+  }
+
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPage.java b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPage.java
index 4d8f381..fd1875e 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPage.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPage.java
@@ -18,16 +18,24 @@
  */
 package org.apache.parquet.column.page;
 
+import java.util.Optional;
+
 /**
  * one data page in a chunk
  */
 abstract public class DataPage extends Page {
 
   private final int valueCount;
+  private final long firstRowIndex;
 
   DataPage(int compressedSize, int uncompressedSize, int valueCount) {
+    this(compressedSize, uncompressedSize, valueCount, -1);
+  }
+
+  DataPage(int compressedSize, int uncompressedSize, int valueCount, long firstRowIndex) {
     super(compressedSize, uncompressedSize);
     this.valueCount = valueCount;
+    this.firstRowIndex = firstRowIndex;
   }
 
   /**
@@ -37,6 +45,20 @@ abstract public class DataPage extends Page {
     return valueCount;
   }
 
+  /**
+   * @return the index of the first row in this page if the related data is available (the optional column-index
+   *         contains this value)
+   */
+  public Optional<Long> getFirstRowIndex() {
+    return firstRowIndex < 0 ? Optional.empty() : Optional.of(firstRowIndex);
+  }
+
+  /**
+   * @return the number of rows in this page if the related data is available (in case of pageV1 the optional
+   *         column-index contains this value)
+   */
+  public abstract Optional<Integer> getIndexRowCount();
+
   public abstract <T> T accept(Visitor<T> visitor);
 
   public static interface Visitor<T> {
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV1.java b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV1.java
index 56928c3..b1f68ae 100755
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV1.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV1.java
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.column.page;
 
+import java.util.Optional;
+
 import org.apache.parquet.Ints;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
@@ -30,6 +32,7 @@ public class DataPageV1 extends DataPage {
   private final Encoding rlEncoding;
   private final Encoding dlEncoding;
   private final Encoding valuesEncoding;
+  private final int indexRowCount;
 
   /**
    * @param bytes the bytes for this page
@@ -47,6 +50,29 @@ public class DataPageV1 extends DataPage {
     this.rlEncoding = rlEncoding;
     this.dlEncoding = dlEncoding;
     this.valuesEncoding = valuesEncoding;
+    this.indexRowCount = -1;
+  }
+
+  /**
+   * @param bytes the bytes for this page
+   * @param valueCount count of values in this page
+   * @param uncompressedSize the uncompressed size of the page
+   * @param firstRowIndex the index of the first row in this page
+   * @param rowCount the number of rows in this page
+   * @param statistics of the page's values (max, min, num_null)
+   * @param rlEncoding the repetition level encoding for this page
+   * @param dlEncoding the definition level encoding for this page
+   * @param valuesEncoding the values encoding for this page
+   */
+  public DataPageV1(BytesInput bytes, int valueCount, int uncompressedSize, long firstRowIndex, int rowCount,
+      Statistics<?> statistics, Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) {
+    super(Ints.checkedCast(bytes.size()), uncompressedSize, valueCount, firstRowIndex);
+    this.bytes = bytes;
+    this.statistics = statistics;
+    this.rlEncoding = rlEncoding;
+    this.dlEncoding = dlEncoding;
+    this.valuesEncoding = valuesEncoding;
+    this.indexRowCount = rowCount;
   }
 
   /**
@@ -94,4 +120,9 @@ public class DataPageV1 extends DataPage {
   public <T> T accept(Visitor<T> visitor) {
     return visitor.visit(this);
   }
+
+  @Override
+  public Optional<Integer> getIndexRowCount() {
+    return indexRowCount < 0 ? Optional.empty() : Optional.of(indexRowCount);
+  }
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV2.java b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV2.java
index 62dac83..a1700ae 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV2.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/DataPageV2.java
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.column.page;
 
+import java.util.Optional;
+
 import org.apache.parquet.Ints;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
@@ -54,6 +56,32 @@ public class DataPageV2 extends DataPage {
    * @param rowCount count of rows
    * @param nullCount count of nulls
    * @param valueCount count of values
+   * @param firstRowIndex the index of the first row in this page
+   * @param repetitionLevels RLE encoded repetition levels
+   * @param definitionLevels RLE encoded definition levels
+   * @param dataEncoding encoding for the data
+   * @param data data encoded with dataEncoding
+   * @param statistics optional statistics for this page
+   * @return an uncompressed page
+   */
+  public static DataPageV2 uncompressed(
+      int rowCount, int nullCount, int valueCount, long firstRowIndex,
+      BytesInput repetitionLevels, BytesInput definitionLevels,
+      Encoding dataEncoding, BytesInput data,
+      Statistics<?> statistics) {
+    return new DataPageV2(
+        rowCount, nullCount, valueCount, firstRowIndex,
+        repetitionLevels, definitionLevels,
+        dataEncoding, data,
+        Ints.checkedCast(repetitionLevels.size() + definitionLevels.size() + data.size()),
+        statistics,
+        false);
+  }
+
+  /**
+   * @param rowCount count of rows
+   * @param nullCount count of nulls
+   * @param valueCount count of values
    * @param repetitionLevels RLE encoded repetition levels
    * @param definitionLevels RLE encoded definition levels
    * @param dataEncoding encoding for the data
@@ -104,6 +132,25 @@ public class DataPageV2 extends DataPage {
     this.isCompressed = isCompressed;
   }
 
+  private DataPageV2(
+      int rowCount, int nullCount, int valueCount, long firstRowIndex,
+      BytesInput repetitionLevels, BytesInput definitionLevels,
+      Encoding dataEncoding, BytesInput data,
+      int uncompressedSize,
+      Statistics<?> statistics,
+      boolean isCompressed) {
+    super(Ints.checkedCast(repetitionLevels.size() + definitionLevels.size() + data.size()), uncompressedSize,
+        valueCount, firstRowIndex);
+    this.rowCount = rowCount;
+    this.nullCount = nullCount;
+    this.repetitionLevels = repetitionLevels;
+    this.definitionLevels = definitionLevels;
+    this.dataEncoding = dataEncoding;
+    this.data = data;
+    this.statistics = statistics;
+    this.isCompressed = isCompressed;
+  }
+
   public int getRowCount() {
     return rowCount;
   }
@@ -137,6 +184,11 @@ public class DataPageV2 extends DataPage {
   }
 
   @Override
+  public Optional<Integer> getIndexRowCount() {
+    return Optional.of(rowCount);
+  }
+
+  @Override
   public <T> T accept(Visitor<T> visitor) {
     return visitor.visit(this);
   }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java b/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java
index 24d5825..753bda8 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/PageReadStore.java
@@ -18,6 +18,8 @@
  */
 package org.apache.parquet.column.page;
 
+import java.util.Optional;
+import java.util.PrimitiveIterator;
 import org.apache.parquet.column.ColumnDescriptor;
 
 /**
@@ -29,7 +31,8 @@ public interface PageReadStore {
 
   /**
    *
-   * @param descriptor the descriptor of the column
+   * @param descriptor
+   *          the descriptor of the column
    * @return the page reader for that column
    */
   PageReader getPageReader(ColumnDescriptor descriptor);
@@ -40,4 +43,14 @@ public interface PageReadStore {
    */
   long getRowCount();
 
+  /**
+   * Returns the indexes of the rows to be read/built if the related data is available. All the rows which index is not
+   * returned shall be skipped.
+   *
+   * @return the optional of the incremental iterator of the row indexes or an empty optional if the related data is not
+   *         available
+   */
+  default Optional<PrimitiveIterator.OfLong> getRowIndexes() {
+    return Optional.empty();
+  }
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
index a2d079f..a72be48 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/page/PageWriter.java
@@ -20,7 +20,6 @@ package org.apache.parquet.column.page;
 
 import java.io.IOException;
 
-import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.bytes.BytesInput;
 import org.apache.parquet.column.Encoding;
 import org.apache.parquet.column.statistics.Statistics;
@@ -39,10 +38,26 @@ public interface PageWriter {
    * @param dlEncoding definition level encoding
    * @param valuesEncoding values encoding
    * @throws IOException if there is an exception while writing page data
+   * @deprecated will be removed in 2.0.0. This method does not support writing column indexes; Use
+   *             {@link #writePage(BytesInput, int, int, Statistics, Encoding, Encoding, Encoding)} instead
    */
+  @Deprecated
   void writePage(BytesInput bytesInput, int valueCount, Statistics<?> statistics, Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) throws IOException;
 
   /**
+   * writes a single page
+   * @param bytesInput the bytes for the page
+   * @param valueCount the number of values in that page
+   * @param rowCount the number of rows in that page
+   * @param statistics the statistics for that page
+   * @param rlEncoding repetition level encoding
+   * @param dlEncoding definition level encoding
+   * @param valuesEncoding values encoding
+   * @throws IOException
+   */
+  void writePage(BytesInput bytesInput, int valueCount, int rowCount, Statistics<?> statistics, Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) throws IOException;
+
+  /**
    * writes a single page in the new format
    * @param rowCount the number of rows in this page
    * @param nullCount the number of null values (out of valueCount)
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
index 1154bc4..06771e9 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/ValuesReader.java
@@ -179,5 +179,17 @@ public abstract class ValuesReader {
    * Skips the next value in the page
    */
   abstract public void skip();
+
+  /**
+   * Skips the next n values in the page
+   *
+   * @param n
+   *          the number of values to be skipped
+   */
+  public void skip(int n) {
+    for (int i = 0; i < n; ++i) {
+      skip();
+    }
+  }
 }
 
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
index c8a80fd..80cfaf2 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesReader.java
@@ -91,6 +91,14 @@ public class DeltaBinaryPackingValuesReader extends ValuesReader {
   }
 
   @Override
+  public void skip(int n) {
+    // checkRead() is invoked before incrementing valuesRead so increase valuesRead size in 2 steps
+    valuesRead += n - 1;
+    checkRead();
+    ++valuesRead;
+  }
+
+  @Override
   public int readInteger() {
     // TODO: probably implement it separately
     return (int) readLong();
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
index 1a2ccb9..4dbbcb5 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/deltalengthbytearray/DeltaLengthByteArrayValuesReader.java
@@ -20,8 +20,6 @@ package org.apache.parquet.column.values.deltalengthbytearray;
 
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
-
 import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.column.values.delta.DeltaBinaryPackingValuesReader;
@@ -64,7 +62,15 @@ public class DeltaLengthByteArrayValuesReader extends ValuesReader {
 
   @Override
   public void skip() {
-    int length = lengthReader.readInteger();
+    skip(1);
+  }
+
+  @Override
+  public void skip(int n) {
+    int length = 0;
+    for (int i = 0; i < n; ++i) {
+      length += lengthReader.readInteger();
+    }
     try {
       in.skipFully(length);
     } catch (IOException e) {
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
index 15ed434..631c908 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/FixedLenByteArrayPlainValuesReader.java
@@ -19,7 +19,6 @@
 package org.apache.parquet.column.values.plain;
 
 import java.io.IOException;
-import java.nio.ByteBuffer;
 import org.apache.parquet.bytes.ByteBufferInputStream;
 import org.apache.parquet.column.values.ValuesReader;
 import org.apache.parquet.io.ParquetDecodingException;
@@ -51,8 +50,13 @@ public class FixedLenByteArrayPlainValuesReader extends ValuesReader {
 
   @Override
   public void skip() {
+    skip(1);
+  }
+
+  @Override
+  public void skip(int n) {
     try {
-      in.skipFully(length);
+      in.skipFully(n * length);
     } catch (IOException | RuntimeException e) {
       throw new ParquetDecodingException("could not skip bytes at offset " + in.position(), e);
     }
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
index f576528..127817e 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/plain/PlainValuesReader.java
@@ -41,14 +41,26 @@ abstract public class PlainValuesReader extends ValuesReader {
     this.in = new LittleEndianDataInputStream(stream.remainingStream());
   }
 
+  @Override
+  public void skip() {
+    skip(1);
+  }
+
+  void skipBytesFully(int n) throws IOException {
+    int skipped = 0;
+    while (skipped < n) {
+      skipped += in.skipBytes(n - skipped);
+    }
+  }
+
   public static class DoublePlainValuesReader extends PlainValuesReader {
 
     @Override
-    public void skip() {
+    public void skip(int n) {
       try {
-        in.skipBytes(8);
+        skipBytesFully(n * 8);
       } catch (IOException e) {
-        throw new ParquetDecodingException("could not skip double", e);
+        throw new ParquetDecodingException("could not skip " + n + " double values", e);
       }
     }
 
@@ -65,11 +77,11 @@ abstract public class PlainValuesReader extends ValuesReader {
   public static class FloatPlainValuesReader extends PlainValuesReader {
 
     @Override
-    public void skip() {
+    public void skip(int n) {
       try {
-        in.skipBytes(4);
+        skipBytesFully(n * 4);
       } catch (IOException e) {
-        throw new ParquetDecodingException("could not skip float", e);
+        throw new ParquetDecodingException("could not skip " + n + " floats", e);
       }
     }
 
@@ -86,11 +98,11 @@ abstract public class PlainValuesReader extends ValuesReader {
   public static class IntegerPlainValuesReader extends PlainValuesReader {
 
     @Override
-    public void skip() {
+    public void skip(int n) {
       try {
-        in.skipBytes(4);
+        in.skipBytes(n * 4);
       } catch (IOException e) {
-        throw new ParquetDecodingException("could not skip int", e);
+        throw new ParquetDecodingException("could not skip " + n + " ints", e);
       }
     }
 
@@ -107,11 +119,11 @@ abstract public class PlainValuesReader extends ValuesReader {
   public static class LongPlainValuesReader extends PlainValuesReader {
 
     @Override
-    public void skip() {
+    public void skip(int n) {
       try {
-        in.skipBytes(8);
+        in.skipBytes(n * 8);
       } catch (IOException e) {
-        throw new ParquetDecodingException("could not skip long", e);
+        throw new ParquetDecodingException("could not skip " + n + " longs", e);
       }
     }
 
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
index 3b7a5de..a51a8c4 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/RunLengthBitPackingHybridValuesWriter.java
@@ -19,6 +19,7 @@
 package org.apache.parquet.column.values.rle;
 
 import java.io.IOException;
+import java.util.Objects;
 
 import org.apache.parquet.bytes.ByteBufferAllocator;
 import org.apache.parquet.Ints;
@@ -28,10 +29,14 @@ import org.apache.parquet.column.values.ValuesWriter;
 import org.apache.parquet.io.ParquetEncodingException;
 
 public class RunLengthBitPackingHybridValuesWriter extends ValuesWriter {
-  private final RunLengthBitPackingHybridEncoder encoder;
+  protected final RunLengthBitPackingHybridEncoder encoder;
 
   public RunLengthBitPackingHybridValuesWriter(int bitWidth, int initialCapacity, int pageSize, ByteBufferAllocator allocator) {
-    this.encoder = new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity, pageSize, allocator);
+    this(new RunLengthBitPackingHybridEncoder(bitWidth, initialCapacity, pageSize, allocator));
+  }
+
+  protected RunLengthBitPackingHybridValuesWriter(RunLengthBitPackingHybridEncoder encoder) {
+    this.encoder = Objects.requireNonNull(encoder);
   }
 
   @Override
diff --git a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/ZeroIntegerValuesReader.java b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/ZeroIntegerValuesReader.java
index beeb0ad..09ca8a1 100644
--- a/parquet-column/src/main/java/org/apache/parquet/column/values/rle/ZeroIntegerValuesReader.java
+++ b/parquet-column/src/main/java/org/apache/parquet/column/values/rle/ZeroIntegerValuesReader.java
@@ -42,4 +42,8 @@ public class ZeroIntegerValuesReader extends ValuesReader {
   public void skip() {
   }
 
+  @Override
+  public void skip(int n) {
+  }
+
 }
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryColumnIndexBuilder.java
new file mode 100644
index 0000000..490cc3e
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryColumnIndexBuilder.java
@@ -0,0 +1,140 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+class BinaryColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class BinaryColumnIndex extends ColumnIndexBase<Binary> {
+    private Binary[] minValues;
+    private Binary[] maxValues;
+
+    private BinaryColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Binary>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final Binary v = (Binary) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final List<Binary> minValues = new ArrayList<>();
+  private final List<Binary> maxValues = new ArrayList<>();
+  private final BinaryTruncator truncator;
+  private final int truncateLength;
+
+  private static Binary convert(ByteBuffer buffer) {
+    return Binary.fromReusedByteBuffer(buffer);
+  }
+
+  private static ByteBuffer convert(Binary value) {
+    return value.toByteBuffer();
+  }
+
+  BinaryColumnIndexBuilder(PrimitiveType type, int truncateLength) {
+    truncator = BinaryTruncator.getTruncator(type);
+    this.truncateLength = truncateLength;
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    minValues.add(min == null ? null : truncator.truncateMin((Binary) min, truncateLength));
+    maxValues.add(max == null ? null : truncator.truncateMax((Binary) max, truncateLength));
+  }
+
+  @Override
+  ColumnIndexBase<Binary> createColumnIndex(PrimitiveType type) {
+    BinaryColumnIndex columnIndex = new BinaryColumnIndex(type);
+    columnIndex.minValues = minValues.toArray(new Binary[minValues.size()]);
+    columnIndex.maxValues = maxValues.toArray(new Binary[maxValues.size()]);
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return ((Binary) value).length();
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryTruncator.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryTruncator.java
new file mode 100644
index 0000000..bcc43fb
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BinaryTruncator.java
@@ -0,0 +1,208 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.nio.charset.Charset;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CoderResult;
+import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.OriginalType;
+import org.apache.parquet.schema.PrimitiveType;
+
+/**
+ * Class for truncating min/max values for binary types.
+ */
+abstract class BinaryTruncator {
+  enum Validity {
+    VALID, MALFORMED, UNMAPPABLE;
+  }
+
+  private static class CharsetValidator {
+    private final CharBuffer dummyBuffer = CharBuffer.allocate(1024);
+    private final CharsetDecoder decoder;
+
+    CharsetValidator(Charset charset) {
+      decoder = charset.newDecoder();
+      decoder.onMalformedInput(CodingErrorAction.REPORT);
+      decoder.onUnmappableCharacter(CodingErrorAction.REPORT);
+    }
+
+    Validity checkValidity(ByteBuffer buffer) {
+      int pos = buffer.position();
+      CoderResult result = CoderResult.OVERFLOW;
+      while (result.isOverflow()) {
+        dummyBuffer.clear();
+        result = decoder.decode(buffer, dummyBuffer, true);
+      }
+      buffer.position(pos);
+      if (result.isUnderflow()) {
+        return Validity.VALID;
+      } else if (result.isMalformed()) {
+        return Validity.MALFORMED;
+      } else {
+        return Validity.UNMAPPABLE;
+      }
+    }
+  }
+
+  private static final BinaryTruncator NO_OP_TRUNCATOR = new BinaryTruncator() {
+    @Override
+    Binary truncateMin(Binary minValue, int length) {
+      return minValue;
+    }
+
+    @Override
+    Binary truncateMax(Binary maxValue, int length) {
+      return maxValue;
+    }
+  };
+
+  private static final BinaryTruncator DEFAULT_UTF8_TRUNCATOR = new BinaryTruncator() {
+    private final CharsetValidator validator = new CharsetValidator(StandardCharsets.UTF_8);
+
+    @Override
+    Binary truncateMin(Binary minValue, int length) {
+      if (minValue.length() <= length) {
+        return minValue;
+      }
+      ByteBuffer buffer = minValue.toByteBuffer();
+      byte[] array;
+      if (validator.checkValidity(buffer) == Validity.VALID) {
+        array = truncateUtf8(buffer, length);
+      } else {
+        array = truncate(buffer, length);
+      }
+      return array == null ? minValue : Binary.fromConstantByteArray(array);
+    }
+
+    @Override
+    Binary truncateMax(Binary maxValue, int length) {
+      if (maxValue.length() <= length) {
+        return maxValue;
+      }
+      byte[] array;
+      ByteBuffer buffer = maxValue.toByteBuffer();
+      if (validator.checkValidity(buffer) == Validity.VALID) {
+        array = incrementUtf8(truncateUtf8(buffer, length));
+      } else {
+        array = increment(truncate(buffer, length));
+      }
+      return array == null ? maxValue : Binary.fromConstantByteArray(array);
+    }
+
+    // Simply truncate to length
+    private byte[] truncate(ByteBuffer buffer, int length) {
+      assert length < buffer.remaining();
+      byte[] array = new byte[length];
+      buffer.get(array);
+      return array;
+    }
+
+    // Trying to increment the bytes from the last one to the beginning
+    private byte[] increment(byte[] array) {
+      for (int i = array.length - 1; i >= 0; --i) {
+        byte elem = array[i];
+        ++elem;
+        array[i] = elem;
+        if (elem != 0) { // Did not overflow: 0xFF -> 0x00
+          return array;
+        }
+      }
+      return null;
+    }
+
+    // Truncates the buffer to length or less so the remaining bytes form a valid UTF-8 string
+    private byte[] truncateUtf8(ByteBuffer buffer, int length) {
+      assert length < buffer.remaining();
+      ByteBuffer newBuffer = buffer.slice();
+      newBuffer.limit(newBuffer.position() + length);
+      while (validator.checkValidity(newBuffer) != Validity.VALID) {
+        newBuffer.limit(newBuffer.limit() - 1);
+        if (newBuffer.remaining() == 0) {
+          return null;
+        }
+      }
+      byte[] array = new byte[newBuffer.remaining()];
+      newBuffer.get(array);
+      return array;
+    }
+
+    // Trying to increment the bytes from the last one to the beginning until the bytes form a valid UTF-8 string
+    private byte[] incrementUtf8(byte[] array) {
+      if (array == null) {
+        return null;
+      }
+      ByteBuffer buffer = ByteBuffer.wrap(array);
+      for (int i = array.length - 1; i >= 0; --i) {
+        byte prev = array[i];
+        byte inc = prev;
+        while (++inc != 0) { // Until overflow: 0xFF -> 0x00
+          array[i] = inc;
+          switch (validator.checkValidity(buffer)) {
+            case VALID:
+              return array;
+            case UNMAPPABLE:
+              continue; // Increment the i byte once more
+            case MALFORMED:
+              break; // Stop incrementing the i byte; go to the i-1
+          }
+          break; // MALFORMED
+        }
+        array[i] = prev;
+      }
+      return null; // All characters are the largest possible; unable to increment
+    }
+  };
+
+  static BinaryTruncator getTruncator(PrimitiveType type) {
+    if (type == null) {
+      return NO_OP_TRUNCATOR;
+    }
+    switch (type.getPrimitiveTypeName()) {
+      case INT96:
+        return NO_OP_TRUNCATOR;
+      case BINARY:
+      case FIXED_LEN_BYTE_ARRAY:
+        OriginalType originalType = type.getOriginalType();
+        if (originalType == null) {
+          return DEFAULT_UTF8_TRUNCATOR;
+        }
+        switch (originalType) {
+          case UTF8:
+          case ENUM:
+          case JSON:
+          case BSON:
+            return DEFAULT_UTF8_TRUNCATOR;
+          default:
+            return NO_OP_TRUNCATOR;
+        }
+      default:
+        throw new IllegalArgumentException("No truncator is available for the type: " + type);
+    }
+  }
+
+  abstract Binary truncateMin(Binary minValue, int length);
+
+  abstract Binary truncateMax(Binary maxValue, int length);
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BooleanColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BooleanColumnIndexBuilder.java
new file mode 100644
index 0000000..233bd1b
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BooleanColumnIndexBuilder.java
@@ -0,0 +1,133 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.nio.ByteBuffer;
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+import it.unimi.dsi.fastutil.booleans.BooleanArrayList;
+import it.unimi.dsi.fastutil.booleans.BooleanList;
+
+class BooleanColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class BooleanColumnIndex extends ColumnIndexBase<Boolean> {
+    private boolean[] minValues;
+    private boolean[] maxValues;
+
+    private BooleanColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Boolean>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final boolean v = (boolean) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final BooleanList minValues = new BooleanArrayList();
+  private final BooleanList maxValues = new BooleanArrayList();
+
+  private static boolean convert(ByteBuffer buffer) {
+    return buffer.get(0) != 0;
+  }
+
+  private static ByteBuffer convert(boolean value) {
+    return ByteBuffer.allocate(1).put(0, value ? (byte) 1 : 0);
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    minValues.add((boolean) min);
+    maxValues.add((boolean) max);
+  }
+
+  @Override
+  ColumnIndexBase<Boolean> createColumnIndex(PrimitiveType type) {
+    BooleanColumnIndex columnIndex = new BooleanColumnIndex(type);
+    columnIndex.minValues = minValues.toBooleanArray();
+    columnIndex.maxValues = maxValues.toBooleanArray();
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return 1;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BoundaryOrder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BoundaryOrder.java
new file mode 100644
index 0000000..e47b5b3
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/BoundaryOrder.java
@@ -0,0 +1,352 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.util.PrimitiveIterator;
+import java.util.PrimitiveIterator.OfInt;
+
+import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder.ColumnIndexBase;
+
+/**
+ * Enum for {@link org.apache.parquet.format.BoundaryOrder}. It also contains the implementations of searching for
+ * matching page indexes for column index based filtering.
+ */
+public enum BoundaryOrder {
+  UNORDERED {
+    @Override
+    PrimitiveIterator.OfInt eq(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMin(arrayIndex) >= 0 && comparator.compareValueToMax(arrayIndex) <= 0,
+          comparator::translate);
+    }
+
+    @Override
+    PrimitiveIterator.OfInt gt(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMax(arrayIndex) < 0,
+          comparator::translate);
+    }
+
+    @Override
+    PrimitiveIterator.OfInt gtEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMax(arrayIndex) <= 0,
+          comparator::translate);
+    }
+
+    @Override
+    PrimitiveIterator.OfInt lt(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMin(arrayIndex) > 0,
+          comparator::translate);
+    }
+
+    @Override
+    PrimitiveIterator.OfInt ltEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMin(arrayIndex) >= 0,
+          comparator::translate);
+    }
+
+    @Override
+    PrimitiveIterator.OfInt notEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      return IndexIterator.filterTranslate(comparator.arrayLength(),
+          arrayIndex -> comparator.compareValueToMin(arrayIndex) != 0 || comparator.compareValueToMax(arrayIndex) != 0,
+          comparator::translate);
+    }
+  },
+  ASCENDING {
+    @Override
+    OfInt eq(ColumnIndexBase<?>.ValueComparator comparator) {
+      Bounds bounds = findBounds(comparator);
+      if (bounds == null) {
+        return IndexIterator.EMPTY;
+      }
+      return IndexIterator.rangeTranslate(bounds.lower, bounds.upper, comparator::translate);
+    }
+
+    @Override
+    OfInt gt(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = 0;
+      int right = length;
+      do {
+        int i = floorMid(left, right);
+        if (comparator.compareValueToMax(i) >= 0) {
+          left = i + 1;
+        } else {
+          right = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(right, length - 1, comparator::translate);
+    }
+
+    @Override
+    OfInt gtEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = 0;
+      int right = length;
+      do {
+        int i = floorMid(left, right);
+        if (comparator.compareValueToMax(i) > 0) {
+          left = i + 1;
+        } else {
+          right = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(right, length - 1, comparator::translate);
+    }
+
+    @Override
+    OfInt lt(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = -1;
+      int right = length - 1;
+      do {
+        int i = ceilingMid(left, right);
+        if (comparator.compareValueToMin(i) <= 0) {
+          right = i - 1;
+        } else {
+          left = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(0, left, comparator::translate);
+    }
+
+    @Override
+    OfInt ltEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = -1;
+      int right = length - 1;
+      do {
+        int i = ceilingMid(left, right);
+        if (comparator.compareValueToMin(i) < 0) {
+          right = i - 1;
+        } else {
+          left = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(0, left, comparator::translate);
+    }
+
+    @Override
+    OfInt notEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      Bounds bounds = findBounds(comparator);
+      int length = comparator.arrayLength();
+      if (bounds == null) {
+        return IndexIterator.all(comparator);
+      }
+      return IndexIterator.filterTranslate(
+          length,
+          i -> i < bounds.lower || i > bounds.upper || comparator.compareValueToMin(i) != 0
+              || comparator.compareValueToMax(i) != 0,
+          comparator::translate);
+    }
+
+    private Bounds findBounds(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int lowerLeft = 0;
+      int upperLeft = 0;
+      int lowerRight = length - 1;
+      int upperRight = length - 1;
+      do {
+        if (lowerLeft > lowerRight) {
+          return null;
+        }
+        int i = floorMid(lowerLeft, lowerRight);
+        if (comparator.compareValueToMin(i) < 0) {
+          lowerRight = upperRight = i - 1;
+        } else if (comparator.compareValueToMax(i) > 0) {
+          lowerLeft = upperLeft = i + 1;
+        } else {
+          lowerRight = upperLeft = i;
+        }
+      } while (lowerLeft != lowerRight);
+      do {
+        if (upperLeft > upperRight) {
+          return null;
+        }
+        int i = ceilingMid(upperLeft, upperRight);
+        if (comparator.compareValueToMin(i) < 0) {
+          upperRight = i - 1;
+        } else if (comparator.compareValueToMax(i) > 0) {
+          upperLeft = i + 1;
+        } else {
+          upperLeft = i;
+        }
+      } while (upperLeft != upperRight);
+      return new Bounds(lowerLeft, upperRight);
+    }
+  },
+  DESCENDING {
+    @Override
+    OfInt eq(ColumnIndexBase<?>.ValueComparator comparator) {
+      Bounds bounds = findBounds(comparator);
+      if (bounds == null) {
+        return IndexIterator.EMPTY;
+      }
+      return IndexIterator.rangeTranslate(bounds.lower, bounds.upper, comparator::translate);
+    }
+
+    @Override
+    OfInt gt(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = -1;
+      int right = length - 1;
+      do {
+        int i = ceilingMid(left, right);
+        if (comparator.compareValueToMax(i) >= 0) {
+          right = i - 1;
+        } else {
+          left = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(0, left, comparator::translate);
+    }
+
+    @Override
+    OfInt gtEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = -1;
+      int right = length - 1;
+      do {
+        int i = ceilingMid(left, right);
+        if (comparator.compareValueToMax(i) > 0) {
+          right = i - 1;
+        } else {
+          left = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(0, left, comparator::translate);
+    }
+
+    @Override
+    OfInt lt(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = 0;
+      int right = length;
+      do {
+        int i = floorMid(left, right);
+        if (comparator.compareValueToMin(i) <= 0) {
+          left = i + 1;
+        } else {
+          right = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(right, length - 1, comparator::translate);
+    }
+
+    @Override
+    OfInt ltEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int left = 0;
+      int right = length;
+      do {
+        int i = floorMid(left, right);
+        if (comparator.compareValueToMin(i) < 0) {
+          left = i + 1;
+        } else {
+          right = i;
+        }
+      } while (left < right);
+      return IndexIterator.rangeTranslate(right, length - 1, comparator::translate);
+    }
+
+    @Override
+    OfInt notEq(ColumnIndexBase<?>.ValueComparator comparator) {
+      Bounds bounds = findBounds(comparator);
+      int length = comparator.arrayLength();
+      if (bounds == null) {
+        return IndexIterator.all(comparator);
+      }
+      return IndexIterator.filterTranslate(
+          length,
+          i -> i < bounds.lower || i > bounds.upper || comparator.compareValueToMin(i) != 0
+              || comparator.compareValueToMax(i) != 0,
+          comparator::translate);
+    }
+
+    private Bounds findBounds(ColumnIndexBase<?>.ValueComparator comparator) {
+      int length = comparator.arrayLength();
+      int lowerLeft = 0;
+      int upperLeft = 0;
+      int lowerRight = length - 1;
+      int upperRight = length - 1;
+      do {
+        if (lowerLeft > lowerRight) {
+          return null;
+        }
+        int i = floorMid(lowerLeft, lowerRight);
+        if (comparator.compareValueToMax(i) > 0) {
+          lowerRight = upperRight = i - 1;
+        } else if (comparator.compareValueToMin(i) < 0) {
+          lowerLeft = upperLeft = i + 1;
+        } else {
+          lowerRight = upperLeft = i;
+        }
+      } while (lowerLeft != lowerRight);
+      do {
+        if (upperLeft > upperRight) {
+          return null;
+        }
+        int i = ceilingMid(upperLeft, upperRight);
+        if (comparator.compareValueToMax(i) > 0) {
+          upperRight = i - 1;
+        } else if (comparator.compareValueToMin(i) < 0) {
+          upperLeft = i + 1;
+        } else {
+          upperLeft = i;
+        }
+      } while (upperLeft != upperRight);
+      return new Bounds(lowerLeft, upperRight);
+    }
+  };
+
+  private static class Bounds {
+    final int lower, upper;
+
+    Bounds(int lower, int upper) {
+      assert lower <= upper;
+      this.lower = lower;
+      this.upper = upper;
+    }
+  }
+
+  private static int floorMid(int left, int right) {
+    // Avoid the possible overflow might happen in case of (left + right) / 2
+    return left + ((right - left) / 2);
+  }
+
+  private static int ceilingMid(int left, int right) {
+    // Avoid the possible overflow might happen in case of (left + right + 1) / 2
+    return left + ((right - left + 1) / 2);
+  }
+
+  abstract PrimitiveIterator.OfInt eq(ColumnIndexBase<?>.ValueComparator comparator);
+
+  abstract PrimitiveIterator.OfInt gt(ColumnIndexBase<?>.ValueComparator comparator);
+
+  abstract PrimitiveIterator.OfInt gtEq(ColumnIndexBase<?>.ValueComparator comparator);
+
+  abstract PrimitiveIterator.OfInt lt(ColumnIndexBase<?>.ValueComparator comparator);
+
+  abstract PrimitiveIterator.OfInt ltEq(ColumnIndexBase<?>.ValueComparator comparator);
+
+  abstract PrimitiveIterator.OfInt notEq(ColumnIndexBase<?>.ValueComparator comparator);
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndex.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndex.java
new file mode 100644
index 0000000..b91a5c0
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndex.java
@@ -0,0 +1,60 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.PrimitiveIterator;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate.Visitor;
+import org.apache.parquet.internal.filter2.columnindex.ColumnIndexFilter;
+
+/**
+ * Column index containing min/max and null count values for the pages in a column chunk. It also implements methods of
+ * {@link Visitor} to return the indexes of the matching pages. They are used by {@link ColumnIndexFilter}.
+ *
+ * @see org.apache.parquet.format.ColumnIndex
+ */
+public interface ColumnIndex extends Visitor<PrimitiveIterator.OfInt> {
+  /**
+   * @return the boundary order of the min/max values; used for converting to the related thrift object
+   */
+  public BoundaryOrder getBoundaryOrder();
+
+  /**
+   * @return the unmodifiable list of null counts; used for converting to the related thrift object
+   */
+  public List<Long> getNullCounts();
+
+  /**
+   * @return the unmodifiable list of null pages; used for converting to the related thrift object
+   */
+  public List<Boolean> getNullPages();
+
+  /**
+   * @return the list of the min values as {@link ByteBuffer}s; used for converting to the related thrift object
+   */
+  public List<ByteBuffer> getMinValues();
+
+  /**
+   * @return the list of the max values as {@link ByteBuffer}s; used for converting to the related thrift object
+   */
+  public List<ByteBuffer> getMaxValues();
+
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
new file mode 100644
index 0000000..b28fdde
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/ColumnIndexBuilder.java
@@ -0,0 +1,636 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.util.Objects.requireNonNull;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.Formatter;
+import java.util.List;
+import java.util.Map;
+import java.util.PrimitiveIterator;
+import java.util.function.IntPredicate;
+
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.filter2.predicate.UserDefinedPredicate;
+import org.apache.parquet.filter2.predicate.Operators.And;
+import org.apache.parquet.filter2.predicate.Operators.Eq;
+import org.apache.parquet.filter2.predicate.Operators.Gt;
+import org.apache.parquet.filter2.predicate.Operators.GtEq;
+import org.apache.parquet.filter2.predicate.Operators.LogicalNotUserDefined;
+import org.apache.parquet.filter2.predicate.Operators.Lt;
+import org.apache.parquet.filter2.predicate.Operators.LtEq;
+import org.apache.parquet.filter2.predicate.Operators.Not;
+import org.apache.parquet.filter2.predicate.Operators.NotEq;
+import org.apache.parquet.filter2.predicate.Operators.Or;
+import org.apache.parquet.filter2.predicate.Operators.UserDefined;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveStringifier;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+
+import it.unimi.dsi.fastutil.booleans.BooleanArrayList;
+import it.unimi.dsi.fastutil.booleans.BooleanList;
+import it.unimi.dsi.fastutil.booleans.BooleanLists;
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import it.unimi.dsi.fastutil.ints.IntOpenHashSet;
+import it.unimi.dsi.fastutil.ints.IntSet;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+import it.unimi.dsi.fastutil.longs.LongLists;
+
+/**
+ * Builder implementation to create {@link ColumnIndex} objects.
+ */
+public abstract class ColumnIndexBuilder {
+
+  static abstract class ColumnIndexBase<C> implements ColumnIndex {
+    /*
+     * A class containing the value to be compared to the min/max values. This way we only need to do the deboxing once
+     * per predicate execution instead for every comparison.
+     */
+    abstract class ValueComparator {
+      abstract int compareValueToMin(int arrayIndex);
+
+      abstract int compareValueToMax(int arrayIndex);
+
+      int arrayLength() {
+        return pageIndexes.length;
+      }
+
+      int translate(int arrayIndex) {
+        return pageIndexes[arrayIndex];
+      }
+    }
+
+    private static final ByteBuffer EMPTY_BYTE_BUFFER = ByteBuffer.allocate(0);
+    private static final int MAX_VALUE_LENGTH_FOR_TOSTRING = 40;
+    private static final String TOSTRING_TRUNCATION_MARKER = "(...)";
+    private static final int TOSTRING_TRUNCATION_START_POS = (MAX_VALUE_LENGTH_FOR_TOSTRING
+        - TOSTRING_TRUNCATION_MARKER.length()) / 2;
+    private static final int TOSTRING_TRUNCATION_END_POS = MAX_VALUE_LENGTH_FOR_TOSTRING
+        - TOSTRING_TRUNCATION_MARKER.length() - TOSTRING_TRUNCATION_START_POS;
+    private static final String TOSTRING_MISSING_VALUE_MARKER = "<none>";
+
+    final PrimitiveStringifier stringifier;
+    final PrimitiveComparator<C> comparator;
+    private boolean[] nullPages;
+    private BoundaryOrder boundaryOrder;
+    // Storing the page index for each array index (min/max values are not stored for null-pages)
+    private int[] pageIndexes;
+    // might be null
+    private long[] nullCounts;
+
+    static String truncate(String str) {
+      if (str.length() <= MAX_VALUE_LENGTH_FOR_TOSTRING) {
+        return str;
+      }
+      return str.substring(0, TOSTRING_TRUNCATION_START_POS) + TOSTRING_TRUNCATION_MARKER
+          + str.substring(str.length() - TOSTRING_TRUNCATION_END_POS);
+    }
+
+    ColumnIndexBase(PrimitiveType type) {
+      comparator = type.comparator();
+      stringifier = type.stringifier();
+    }
+
+    @Override
+    public BoundaryOrder getBoundaryOrder() {
+      return boundaryOrder;
+    }
+
+    @Override
+    public List<Long> getNullCounts() {
+      if (nullCounts == null) {
+        return null;
+      }
+      return LongLists.unmodifiable(LongArrayList.wrap(nullCounts));
+    }
+
+    @Override
+    public List<Boolean> getNullPages() {
+      return BooleanLists.unmodifiable(BooleanArrayList.wrap(nullPages));
+    }
+
+    @Override
+    public List<ByteBuffer> getMinValues() {
+      List<ByteBuffer> list = new ArrayList<>(getPageCount());
+      int arrayIndex = 0;
+      for (int i = 0, n = getPageCount(); i < n; ++i) {
+        if (isNullPage(i)) {
+          list.add(EMPTY_BYTE_BUFFER);
+        } else {
+          list.add(getMinValueAsBytes(arrayIndex++));
+        }
+      }
+      return list;
+    }
+
+    @Override
+    public List<ByteBuffer> getMaxValues() {
+      List<ByteBuffer> list = new ArrayList<>(getPageCount());
+      int arrayIndex = 0;
+      for (int i = 0, n = getPageCount(); i < n; ++i) {
+        if (isNullPage(i)) {
+          list.add(EMPTY_BYTE_BUFFER);
+        } else {
+          list.add(getMaxValueAsBytes(arrayIndex++));
+        }
+      }
+      return list;
+    }
+
+    @Override
+    public String toString() {
+      try (Formatter formatter = new Formatter()) {
+        formatter.format("Boudary order: %s\n", boundaryOrder);
+        String minMaxPart = "  %-" + MAX_VALUE_LENGTH_FOR_TOSTRING + "s  %-" + MAX_VALUE_LENGTH_FOR_TOSTRING + "s\n";
+        formatter.format("%-10s  %20s" + minMaxPart, "", "null count", "min", "max");
+        String format = "page-%-5d  %20s" + minMaxPart;
+        int arrayIndex = 0;
+        for (int i = 0, n = nullPages.length; i < n; ++i) {
+          String nullCount = nullCounts == null ? TOSTRING_MISSING_VALUE_MARKER : Long.toString(nullCounts[i]);
+          String min, max;
+          if (nullPages[i]) {
+            min = max = TOSTRING_MISSING_VALUE_MARKER;
+          } else {
+            min = truncate(getMinValueAsString(arrayIndex));
+            max = truncate(getMaxValueAsString(arrayIndex++));
+          }
+          formatter.format(format, i, nullCount, min, max);
+        }
+        return formatter.toString();
+      }
+    }
+
+    int getPageCount() {
+      return nullPages.length;
+    }
+
+    boolean isNullPage(int pageIndex) {
+      return nullPages[pageIndex];
+    }
+
+    /*
+     * Returns the min value for arrayIndex as a ByteBuffer. (Min values are not stored for null-pages so arrayIndex
+     * might not equal to pageIndex.)
+     */
+    abstract ByteBuffer getMinValueAsBytes(int arrayIndex);
+
+    /*
+     * Returns the max value for arrayIndex as a ByteBuffer. (Max values are not stored for null-pages so arrayIndex
+     * might not equal to pageIndex.)
+     */
+    abstract ByteBuffer getMaxValueAsBytes(int arrayIndex);
+
+    /*
+     * Returns the min value for arrayIndex as a String. (Min values are not stored for null-pages so arrayIndex might
+     * not equal to pageIndex.)
+     */
+    abstract String getMinValueAsString(int arrayIndex);
+
+    /*
+     * Returns the max value for arrayIndex as a String. (Max values are not stored for null-pages so arrayIndex might
+     * not equal to pageIndex.)
+     */
+    abstract String getMaxValueAsString(int arrayIndex);
+
+    /* Creates a Statistics object for filtering. Used for user defined predicates. */
+    abstract <T extends Comparable<T>> org.apache.parquet.filter2.predicate.Statistics<T> createStats(int arrayIndex);
+
+    /* Creates a ValueComparator object containing the specified value to be compared for min/max values */
+    abstract ValueComparator createValueComparator(Object value);
+
+    @Override
+    public PrimitiveIterator.OfInt visit(And and) {
+      throw new UnsupportedOperationException("AND shall not be used on column index directly");
+    }
+
+    @Override
+    public PrimitiveIterator.OfInt visit(Not not) {
+      throw new UnsupportedOperationException("NOT shall not be used on column index directly");
+    }
+
+    @Override
+    public PrimitiveIterator.OfInt visit(Or or) {
+      throw new UnsupportedOperationException("OR shall not be used on column index directly");
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(Eq<T> eq) {
+      T value = eq.getValue();
+      if (value == null) {
+        if (nullCounts == null) {
+          // Searching for nulls so if we don't have null related statistics we have to return all pages
+          return IndexIterator.all(getPageCount());
+        } else {
+          return IndexIterator.filter(getPageCount(), pageIndex -> nullCounts[pageIndex] > 0);
+        }
+      }
+      return getBoundaryOrder().eq(createValueComparator(value));
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(Gt<T> gt) {
+      return getBoundaryOrder().gt(createValueComparator(gt.getValue()));
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(GtEq<T> gtEq) {
+      return getBoundaryOrder().gtEq(createValueComparator(gtEq.getValue()));
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(Lt<T> lt) {
+      return getBoundaryOrder().lt(createValueComparator(lt.getValue()));
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(LtEq<T> ltEq) {
+      return getBoundaryOrder().ltEq(createValueComparator(ltEq.getValue()));
+    }
+
+    @Override
+    public <T extends Comparable<T>> PrimitiveIterator.OfInt visit(NotEq<T> notEq) {
+      T value = notEq.getValue();
+      if (value == null) {
+        return IndexIterator.filter(getPageCount(), pageIndex -> !nullPages[pageIndex]);
+      }
+
+      if (nullCounts == null) {
+        // Nulls match so if we don't have null related statistics we have to return all pages
+        return IndexIterator.all(getPageCount());
+      }
+
+      // Merging value filtering with pages containing nulls
+      IntSet matchingIndexes = new IntOpenHashSet();
+      getBoundaryOrder().notEq(createValueComparator(value))
+          .forEachRemaining((int index) -> matchingIndexes.add(index));
+      return IndexIterator.filter(getPageCount(),
+          pageIndex -> nullCounts[pageIndex] > 0 || matchingIndexes.contains(pageIndex));
+    }
+
+    @Override
+    public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> PrimitiveIterator.OfInt visit(
+        UserDefined<T, U> udp) {
+      final UserDefinedPredicate<T> predicate = udp.getUserDefinedPredicate();
+      final boolean acceptNulls = predicate.keep(null);
+
+      if (acceptNulls && nullCounts == null) {
+        // Nulls match so if we don't have null related statistics we have to return all pages
+        return IndexIterator.all(getPageCount());
+      }
+
+      return IndexIterator.filter(getPageCount(), new IntPredicate() {
+        private int arrayIndex = -1;
+
+        @Override
+        public boolean test(int pageIndex) {
+          if (isNullPage(pageIndex)) {
+            return acceptNulls;
+          } else {
+            ++arrayIndex;
+            if (acceptNulls && nullCounts[pageIndex] > 0) {
+              return true;
+            }
+            org.apache.parquet.filter2.predicate.Statistics<T> stats = createStats(arrayIndex);
+            return !predicate.canDrop(stats);
+          }
+        }
+      });
+    }
+
+    @Override
+    public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> PrimitiveIterator.OfInt visit(
+        LogicalNotUserDefined<T, U> udp) {
+      final UserDefinedPredicate<T> inversePredicate = udp.getUserDefined().getUserDefinedPredicate();
+      final boolean acceptNulls = !inversePredicate.keep(null);
+
+      if (acceptNulls && nullCounts == null) {
+        // Nulls match so if we don't have null related statistics we have to return all pages
+        return IndexIterator.all(getPageCount());
+      }
+
+      return IndexIterator.filter(getPageCount(), new IntPredicate() {
+        private int arrayIndex = -1;
+
+        @Override
+        public boolean test(int pageIndex) {
+          if (isNullPage(pageIndex)) {
+            return acceptNulls;
+          } else {
+            ++arrayIndex;
+            if (acceptNulls && nullCounts[pageIndex] > 0) {
+              return true;
+            }
+            org.apache.parquet.filter2.predicate.Statistics<T> stats = createStats(arrayIndex);
+            return !inversePredicate.inverseCanDrop(stats);
+          }
+        }
+      });
+    }
+  }
+
+  private static final ColumnIndexBuilder NO_OP_BUILDER = new ColumnIndexBuilder() {
+    @Override
+    public ColumnIndex build() {
+      return null;
+    }
+
+    @Override
+    public void add(Statistics<?> stats) {
+    }
+
+    @Override
+    void addMinMax(Object min, Object max) {
+    }
+
+    @Override
+    ColumnIndexBase<?> createColumnIndex(PrimitiveType type) {
+      return null;
+    }
+
+    @Override
+    void clearMinMax() {
+    }
+
+    @Override
+    void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    }
+
+    @Override
+    int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+      return 0;
+    }
+
+    @Override
+    int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+      return 0;
+    }
+
+    @Override
+    int sizeOf(Object value) {
+      return 0;
+    }
+  };
+
+  private static final Map<PrimitiveTypeName, ColumnIndexBuilder> BUILDERS = new EnumMap<>(PrimitiveTypeName.class);
+
+  private PrimitiveType type;
+  private final BooleanList nullPages = new BooleanArrayList();
+  private final LongList nullCounts = new LongArrayList();
+  private long minMaxSize;
+  private final IntList pageIndexes = new IntArrayList();
+  private int nextPageIndex;
+
+  /**
+   * @return a no-op builder that does not collect statistics objects and therefore returns {@code null} at
+   *         {@link #build()}.
+   */
+  public static ColumnIndexBuilder getNoOpBuilder() {
+    return NO_OP_BUILDER;
+  }
+
+  /**
+   * @param type
+   *          the type this builder is to be created for
+   * @param truncateLength
+   *          the length to be used for truncating binary values if possible
+   * @return a {@link ColumnIndexBuilder} instance to be used for creating {@link ColumnIndex} objects
+   */
+  public static ColumnIndexBuilder getBuilder(PrimitiveType type, int truncateLength) {
+    ColumnIndexBuilder builder = createNewBuilder(type, truncateLength);
+    builder.type = type;
+    return builder;
+  }
+
+  private static ColumnIndexBuilder createNewBuilder(PrimitiveType type, int truncateLength) {
+    switch (type.getPrimitiveTypeName()) {
+      case BINARY:
+      case FIXED_LEN_BYTE_ARRAY:
+      case INT96:
+        return new BinaryColumnIndexBuilder(type, truncateLength);
+      case BOOLEAN:
+        return new BooleanColumnIndexBuilder();
+      case DOUBLE:
+        return new DoubleColumnIndexBuilder();
+      case FLOAT:
+        return new FloatColumnIndexBuilder();
+      case INT32:
+        return new IntColumnIndexBuilder();
+      case INT64:
+        return new LongColumnIndexBuilder();
+      default:
+        throw new IllegalArgumentException("Unsupported type for column index: " + type);
+    }
+  }
+
+  /**
+   * @param type
+   *          the primitive type
+   * @param boundaryOrder
+   *          the boundary order of the min/max values
+   * @param nullPages
+   *          the null pages (one boolean value for each page that signifies whether the page consists of nulls
+   *          entirely)
+   * @param nullCounts
+   *          the number of null values for each page
+   * @param minValues
+   *          the min values for each page
+   * @param maxValues
+   *          the max values for each page
+   * @return the newly created {@link ColumnIndex} object based on the specified arguments
+   */
+  public static ColumnIndex build(
+      PrimitiveType type,
+      BoundaryOrder boundaryOrder,
+      List<Boolean> nullPages,
+      List<Long> nullCounts,
+      List<ByteBuffer> minValues,
+      List<ByteBuffer> maxValues) {
+
+    PrimitiveTypeName typeName = type.getPrimitiveTypeName();
+    ColumnIndexBuilder builder = BUILDERS.get(typeName);
+    if (builder == null) {
+      builder = createNewBuilder(type, Integer.MAX_VALUE);
+      BUILDERS.put(typeName, builder);
+    }
+
+    builder.fill(nullPages, nullCounts, minValues, maxValues);
+    ColumnIndexBase<?> columnIndex = builder.build(type);
+    columnIndex.boundaryOrder = requireNonNull(boundaryOrder);
+    return columnIndex;
+  }
+
+  ColumnIndexBuilder() {
+    // Shall be able to be created inside this package only
+  }
+
+  /**
+   * Adds the data from the specified statistics to this builder
+   *
+   * @param stats
+   *          the statistics to be added
+   */
+  public void add(Statistics<?> stats) {
+    if (stats.hasNonNullValue()) {
+      nullPages.add(false);
+      Object min = stats.genericGetMin();
+      Object max = stats.genericGetMax();
+      addMinMax(min, max);
+      pageIndexes.add(nextPageIndex);
+      minMaxSize += sizeOf(min);
+      minMaxSize += sizeOf(max);
+    } else {
+      nullPages.add(true);
+    }
+    nullCounts.add(stats.getNumNulls());
+    ++nextPageIndex;
+  }
+
+  abstract void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max);
+
+  abstract void addMinMax(Object min, Object max);
+
+  private void fill(List<Boolean> nullPages, List<Long> nullCounts, List<ByteBuffer> minValues,
+      List<ByteBuffer> maxValues) {
+    clear();
+    int pageCount = nullPages.size();
+    if ((nullCounts != null && nullCounts.size() != pageCount) || minValues.size() != pageCount
+        || maxValues.size() != pageCount) {
+      throw new IllegalArgumentException(
+          String.format("Not all sizes are equal (nullPages:%d, nullCounts:%s, minValues:%d, maxValues:%d",
+              nullPages.size(), nullCounts == null ? "null" : nullCounts.size(), minValues.size(), maxValues.size()));
+    }
+    this.nullPages.addAll(nullPages);
+    // Nullcounts is optional in the format
+    if (nullCounts != null) {
+      this.nullCounts.addAll(nullCounts);
+    }
+
+    for (int i = 0; i < pageCount; ++i) {
+      if (!nullPages.get(i)) {
+        ByteBuffer min = minValues.get(i);
+        ByteBuffer max = maxValues.get(i);
+        addMinMaxFromBytes(min, max);
+        pageIndexes.add(i);
+        minMaxSize += min.remaining();
+        minMaxSize += max.remaining();
+      }
+    }
+  }
+
+  /**
+   * @return the newly created column index or {@code null} if the {@link ColumnIndex} would be empty
+   */
+  public ColumnIndex build() {
+    ColumnIndexBase<?> columnIndex = build(type);
+    if (columnIndex == null) {
+      return null;
+    }
+    columnIndex.boundaryOrder = calculateBoundaryOrder(type.comparator());
+    return columnIndex;
+  }
+
+  private ColumnIndexBase<?> build(PrimitiveType type) {
+    if (nullPages.isEmpty()) {
+      return null;
+    }
+    ColumnIndexBase<?> columnIndex = createColumnIndex(type);
+    if (columnIndex == null) {
+      // Might happen if the specialized builder discovers invalid min/max values
+      return null;
+    }
+    columnIndex.nullPages = nullPages.toBooleanArray();
+    // Null counts is optional so keep it null if the builder has no values
+    if (!nullCounts.isEmpty()) {
+      columnIndex.nullCounts = nullCounts.toLongArray();
+    }
+    columnIndex.pageIndexes = pageIndexes.toIntArray();
+
+    return columnIndex;
+  }
+
+  private BoundaryOrder calculateBoundaryOrder(PrimitiveComparator<Binary> comparator) {
+    if (isAscending(comparator)) {
+      return BoundaryOrder.ASCENDING;
+    } else if (isDescending(comparator)) {
+      return BoundaryOrder.DESCENDING;
+    } else {
+      return BoundaryOrder.UNORDERED;
+    }
+  }
+
+  // min[i] <= min[i+1] && max[i] <= max[i+1]
+  private boolean isAscending(PrimitiveComparator<Binary> comparator) {
+    for (int i = 1, n = pageIndexes.size(); i < n; ++i) {
+      if (compareMinValues(comparator, i - 1, i) > 0 || compareMaxValues(comparator, i - 1, i) > 0) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  // min[i] >= min[i+1] && max[i] >= max[i+1]
+  private boolean isDescending(PrimitiveComparator<Binary> comparator) {
+    for (int i = 1, n = pageIndexes.size(); i < n; ++i) {
+      if (compareMinValues(comparator, i - 1, i) < 0 || compareMaxValues(comparator, i - 1, i) < 0) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  abstract int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2);
+
+  abstract int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2);
+
+  private void clear() {
+    nullPages.clear();
+    nullCounts.clear();
+    clearMinMax();
+    minMaxSize = 0;
+    nextPageIndex = 0;
+    pageIndexes.clear();
+  }
+
+  abstract void clearMinMax();
+
+  abstract ColumnIndexBase<?> createColumnIndex(PrimitiveType type);
+
+  abstract int sizeOf(Object value);
+
+  /**
+   * @return the number of pages added so far to this builder
+   */
+  public int getPageCount() {
+    return nullPages.size();
+  }
+
+  /**
+   * @return the sum of size in bytes of the min/max values added so far to this builder
+   */
+  public long getMinMaxSize() {
+    return minMaxSize;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/DoubleColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/DoubleColumnIndexBuilder.java
new file mode 100644
index 0000000..074d025
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/DoubleColumnIndexBuilder.java
@@ -0,0 +1,155 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.nio.ByteOrder.LITTLE_ENDIAN;
+
+import java.nio.ByteBuffer;
+
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+import it.unimi.dsi.fastutil.doubles.DoubleArrayList;
+import it.unimi.dsi.fastutil.doubles.DoubleList;
+
+class DoubleColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class DoubleColumnIndex extends ColumnIndexBase<Double> {
+    private double[] minValues;
+    private double[] maxValues;
+
+    private DoubleColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Double>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final double v = (double) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final DoubleList minValues = new DoubleArrayList();
+  private final DoubleList maxValues = new DoubleArrayList();
+  private boolean invalid;
+
+  private static double convert(ByteBuffer buffer) {
+    return buffer.order(LITTLE_ENDIAN).getDouble(0);
+  }
+
+  private static ByteBuffer convert(double value) {
+    return ByteBuffer.allocate(Double.BYTES).order(LITTLE_ENDIAN).putDouble(0, value);
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    double dMin = (double) min;
+    double dMax = (double) max;
+    if (Double.isNaN(dMin) || Double.isNaN(dMax)) {
+      // Invalidate this column index in case of NaN as the sorting order of values is undefined for this case
+      invalid = true;
+    }
+
+    // Sorting order is undefined for -0.0 so let min = -0.0 and max = +0.0 to ensure that no 0.0 values are skipped
+    if (Double.compare(dMin, +0.0) == 0) {
+      dMin = -0.0;
+    }
+    if (Double.compare(dMax, -0.0) == 0) {
+      dMax = +0.0;
+    }
+
+    minValues.add(dMin);
+    maxValues.add(dMax);
+  }
+
+  @Override
+  ColumnIndexBase<Double> createColumnIndex(PrimitiveType type) {
+    if (invalid) {
+      return null;
+    }
+    DoubleColumnIndex columnIndex = new DoubleColumnIndex(type);
+    columnIndex.minValues = minValues.toDoubleArray();
+    columnIndex.maxValues = maxValues.toDoubleArray();
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return Double.BYTES;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/FloatColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/FloatColumnIndexBuilder.java
new file mode 100644
index 0000000..cbcdf94
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/FloatColumnIndexBuilder.java
@@ -0,0 +1,155 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.nio.ByteOrder.LITTLE_ENDIAN;
+
+import java.nio.ByteBuffer;
+
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+import it.unimi.dsi.fastutil.floats.FloatArrayList;
+import it.unimi.dsi.fastutil.floats.FloatList;
+
+class FloatColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class FloatColumnIndex extends ColumnIndexBase<Float> {
+    private float[] minValues;
+    private float[] maxValues;
+
+    private FloatColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Float>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final float v = (float) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final FloatList minValues = new FloatArrayList();
+  private final FloatList maxValues = new FloatArrayList();
+  private boolean invalid;
+
+  private static float convert(ByteBuffer buffer) {
+    return buffer.order(LITTLE_ENDIAN).getFloat(0);
+  }
+
+  private static ByteBuffer convert(float value) {
+    return ByteBuffer.allocate(Float.BYTES).order(LITTLE_ENDIAN).putFloat(0, value);
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    float fMin = (float) min;
+    float fMax = (float) max;
+    if (Float.isNaN(fMin) || Float.isNaN(fMax)) {
+      // Invalidate this column index in case of NaN as the sorting order of values is undefined for this case
+      invalid = true;
+    }
+
+    // Sorting order is undefined for -0.0 so let min = -0.0 and max = +0.0 to ensure that no 0.0 values are skipped
+    if (Float.compare(fMin, +0.0f) == 0) {
+      fMin = -0.0f;
+    }
+    if (Float.compare(fMax, -0.0f) == 0) {
+      fMax = +0.0f;
+    }
+
+    minValues.add(fMin);
+    maxValues.add(fMax);
+  }
+
+  @Override
+  ColumnIndexBase<Float> createColumnIndex(PrimitiveType type) {
+    if (invalid) {
+      return null;
+    }
+    FloatColumnIndex columnIndex = new FloatColumnIndex(type);
+    columnIndex.minValues = minValues.toFloatArray();
+    columnIndex.maxValues = maxValues.toFloatArray();
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return Float.BYTES;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IndexIterator.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IndexIterator.java
new file mode 100644
index 0000000..9eab65e
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IndexIterator.java
@@ -0,0 +1,98 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.util.NoSuchElementException;
+import java.util.PrimitiveIterator;
+import java.util.function.IntPredicate;
+import java.util.function.IntUnaryOperator;
+
+import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder.ColumnIndexBase;
+
+/**
+ * Iterator implementation for page indexes.
+ */
+class IndexIterator implements PrimitiveIterator.OfInt {
+  public static final PrimitiveIterator.OfInt EMPTY = new OfInt() {
+    @Override
+    public boolean hasNext() {
+      return false;
+    }
+
+    @Override
+    public int nextInt() {
+      throw new NoSuchElementException();
+    }
+  };
+  private int index;
+  private final int endIndex;
+  private final IntPredicate filter;
+  private final IntUnaryOperator translator;
+
+  static PrimitiveIterator.OfInt all(int pageCount) {
+    return new IndexIterator(0, pageCount, i -> true, i -> i);
+  }
+
+  static PrimitiveIterator.OfInt all(ColumnIndexBase<?>.ValueComparator comparator) {
+    return new IndexIterator(0, comparator.arrayLength(), i -> true, comparator::translate);
+  }
+
+  static PrimitiveIterator.OfInt filter(int pageCount, IntPredicate filter) {
+    return new IndexIterator(0, pageCount, filter, i -> i);
+  }
+
+  static PrimitiveIterator.OfInt filterTranslate(int arrayLength, IntPredicate filter, IntUnaryOperator translator) {
+    return new IndexIterator(0, arrayLength, filter, translator);
+  }
+
+  static PrimitiveIterator.OfInt rangeTranslate(int from, int to, IntUnaryOperator translator) {
+    return new IndexIterator(from, to + 1, i -> true, translator);
+  }
+
+  private IndexIterator(int startIndex, int endIndex, IntPredicate filter, IntUnaryOperator translator) {
+    this.endIndex = endIndex;
+    this.filter = filter;
+    this.translator = translator;
+    index = nextPageIndex(startIndex);
+  }
+
+  private int nextPageIndex(int startIndex) {
+    for (int i = startIndex; i < endIndex; ++i) {
+      if (filter.test(i)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+  @Override
+  public boolean hasNext() {
+    return index >= 0;
+  }
+
+  @Override
+  public int nextInt() {
+    if (hasNext()) {
+      int ret = index;
+      index = nextPageIndex(index + 1);
+      return translator.applyAsInt(ret);
+    }
+    throw new NoSuchElementException();
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IntColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IntColumnIndexBuilder.java
new file mode 100644
index 0000000..2d19d27
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/IntColumnIndexBuilder.java
@@ -0,0 +1,136 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.nio.ByteOrder.LITTLE_ENDIAN;
+
+import java.nio.ByteBuffer;
+
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+class IntColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class IntColumnIndex extends ColumnIndexBase<Integer> {
+    private int[] minValues;
+    private int[] maxValues;
+
+    private IntColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Integer>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final int v = (int) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final IntList minValues = new IntArrayList();
+  private final IntList maxValues = new IntArrayList();
+
+  private static int convert(ByteBuffer buffer) {
+    return buffer.order(LITTLE_ENDIAN).getInt(0);
+  }
+
+  private static ByteBuffer convert(int value) {
+    return ByteBuffer.allocate(Integer.BYTES).order(LITTLE_ENDIAN).putInt(0, value);
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    minValues.add((int) min);
+    maxValues.add((int) max);
+  }
+
+  @Override
+  ColumnIndexBase<Integer> createColumnIndex(PrimitiveType type) {
+    IntColumnIndex columnIndex = new IntColumnIndex(type);
+    columnIndex.minValues = minValues.toIntArray();
+    columnIndex.maxValues = maxValues.toIntArray();
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return Integer.BYTES;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/LongColumnIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/LongColumnIndexBuilder.java
new file mode 100644
index 0000000..b0189b7
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/LongColumnIndexBuilder.java
@@ -0,0 +1,136 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.nio.ByteOrder.LITTLE_ENDIAN;
+
+import java.nio.ByteBuffer;
+
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveComparator;
+import org.apache.parquet.schema.PrimitiveType;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
+class LongColumnIndexBuilder extends ColumnIndexBuilder {
+  private static class LongColumnIndex extends ColumnIndexBase<Long> {
+    private long[] minValues;
+    private long[] maxValues;
+
+    private LongColumnIndex(PrimitiveType type) {
+      super(type);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int pageIndex) {
+      return convert(minValues[pageIndex]);
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int pageIndex) {
+      return convert(maxValues[pageIndex]);
+    }
+
+    @Override
+    String getMinValueAsString(int pageIndex) {
+      return stringifier.stringify(minValues[pageIndex]);
+    }
+
+    @Override
+    String getMaxValueAsString(int pageIndex) {
+      return stringifier.stringify(maxValues[pageIndex]);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    <T extends Comparable<T>> Statistics<T> createStats(int arrayIndex) {
+      return (Statistics<T>) new Statistics<Long>(minValues[arrayIndex], maxValues[arrayIndex], comparator);
+    }
+
+    @Override
+    ValueComparator createValueComparator(Object value) {
+      final long v = (long) value;
+      return new ValueComparator() {
+        @Override
+        int compareValueToMin(int arrayIndex) {
+          return comparator.compare(v, minValues[arrayIndex]);
+        }
+
+        @Override
+        int compareValueToMax(int arrayIndex) {
+          return comparator.compare(v, maxValues[arrayIndex]);
+        }
+      };
+    }
+  }
+
+  private final LongList minValues = new LongArrayList();
+  private final LongList maxValues = new LongArrayList();
+
+  private static long convert(ByteBuffer buffer) {
+    return buffer.order(LITTLE_ENDIAN).getLong(0);
+  }
+
+  private static ByteBuffer convert(long value) {
+    return ByteBuffer.allocate(Long.BYTES).order(LITTLE_ENDIAN).putLong(0, value);
+  }
+
+  @Override
+  void addMinMaxFromBytes(ByteBuffer min, ByteBuffer max) {
+    minValues.add(convert(min));
+    maxValues.add(convert(max));
+  }
+
+  @Override
+  void addMinMax(Object min, Object max) {
+    minValues.add((long) min);
+    maxValues.add((long) max);
+  }
+
+  @Override
+  ColumnIndexBase<Long> createColumnIndex(PrimitiveType type) {
+    LongColumnIndex columnIndex = new LongColumnIndex(type);
+    columnIndex.minValues = minValues.toLongArray();
+    columnIndex.maxValues = maxValues.toLongArray();
+    return columnIndex;
+  }
+
+  @Override
+  void clearMinMax() {
+    minValues.clear();
+    maxValues.clear();
+  }
+
+  @Override
+  int compareMinValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(minValues.get(index1), minValues.get(index2));
+  }
+
+  @Override
+  int compareMaxValues(PrimitiveComparator<Binary> comparator, int index1, int index2) {
+    return comparator.compare(maxValues.get(index1), maxValues.get(index2));
+  }
+
+  @Override
+  int sizeOf(Object value) {
+    return Long.BYTES;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndex.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndex.java
new file mode 100644
index 0000000..ba984eb
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndex.java
@@ -0,0 +1,64 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+/**
+ * Offset index containing the offset and size of the page and the index of the first row in the page.
+ *
+ * @see org.apache.parquet.format.OffsetIndex
+ */
+public interface OffsetIndex {
+  /**
+   * @return the number of pages
+   */
+  public int getPageCount();
+
+  /**
+   * @param pageIndex
+   *          the index of the page
+   * @return the offset of the page in the file
+   */
+  public long getOffset(int pageIndex);
+
+  /**
+   * @param pageIndex
+   *          the index of the page
+   * @return the compressed size of the page (including page header)
+   */
+  public int getCompressedPageSize(int pageIndex);
+
+  /**
+   * @param pageIndex
+   *          the index of the page
+   * @return the index of the first row in the page
+   */
+  public long getFirstRowIndex(int pageIndex);
+
+  /**
+   * @param pageIndex
+   *          the index of the page
+   * @param rowGroupRowCount
+   *          the total number of rows in the row-group
+   * @return the calculated index of the last row of the given page
+   */
+  public default long getLastRowIndex(int pageIndex, long rowGroupRowCount) {
+    int nextPageIndex = pageIndex + 1;
+    return (nextPageIndex >= getPageCount() ? rowGroupRowCount : getFirstRowIndex(nextPageIndex)) - 1;
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndexBuilder.java b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndexBuilder.java
new file mode 100644
index 0000000..e4907b5
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/column/columnindex/OffsetIndexBuilder.java
@@ -0,0 +1,175 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.util.Formatter;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
+/**
+ * Builder implementation to create {@link OffsetIndex} objects during writing a parquet file.
+ */
+public class OffsetIndexBuilder {
+
+  private static class OffsetIndexImpl implements OffsetIndex {
+    private long[] offsets;
+    private int[] compressedPageSizes;
+    private long[] firstRowIndexes;
+
+    @Override
+    public String toString() {
+      try (Formatter formatter = new Formatter()) {
+        formatter.format("%-10s  %20s  %16s  %20s\n", "", "offset", "compressed size", "first row index");
+        for (int i = 0, n = offsets.length; i < n; ++i) {
+          formatter.format("page-%-5d  %20d  %16d  %20d\n", i, offsets[i], compressedPageSizes[i], firstRowIndexes[i]);
+        }
+        return formatter.toString();
+      }
+    }
+
+    @Override
+    public int getPageCount() {
+      return offsets.length;
+    }
+
+    @Override
+    public long getOffset(int pageIndex) {
+      return offsets[pageIndex];
+    }
+
+    @Override
+    public int getCompressedPageSize(int pageIndex) {
+      return compressedPageSizes[pageIndex];
+    }
+
+    @Override
+    public long getFirstRowIndex(int pageIndex) {
+      return firstRowIndexes[pageIndex];
+    }
+  }
+
+  private static final OffsetIndexBuilder NO_OP_BUILDER = new OffsetIndexBuilder() {
+    @Override
+    public void add(int compressedPageSize, long rowCount) {
+    }
+
+    @Override
+    public void add(long offset, int compressedPageSize, long rowCount) {
+    }
+  };
+
+  private final LongList offsets = new LongArrayList();
+  private final IntList compressedPageSizes = new IntArrayList();
+  private final LongList firstRowIndexes = new LongArrayList();
+  private long previousOffset;
+  private int previousPageSize;
+  private long previousRowIndex;
+  private long previousRowCount;
+
+  /**
+   * @return a no-op builder that does not collect values and therefore returns {@code null} at {@link #build(long)}
+   */
+  public static OffsetIndexBuilder getNoOpBuilder() {
+    return NO_OP_BUILDER;
+  }
+
+  /**
+   * @return an {@link OffsetIndexBuilder} instance to build an {@link OffsetIndex} object
+   */
+  public static OffsetIndexBuilder getBuilder() {
+    return new OffsetIndexBuilder();
+  }
+
+  private OffsetIndexBuilder() {
+  }
+
+  /**
+   * Adds the specified parameters to this builder. Used by the writers to building up {@link OffsetIndex} objects to be
+   * written to the Parquet file.
+   *
+   * @param compressedPageSize
+   *          the size of the page (including header)
+   * @param rowCount
+   *          the number of rows in the page
+   */
+  public void add(int compressedPageSize, long rowCount) {
+    add(previousOffset + previousPageSize, compressedPageSize, previousRowIndex + previousRowCount);
+    previousRowCount = rowCount;
+  }
+
+  /**
+   * Adds the specified parameters to this builder. Used by the metadata converter to building up {@link OffsetIndex}
+   * objects read from the Parquet file.
+   *
+   * @param offset
+   *          the offset of the page in the file
+   * @param compressedPageSize
+   *          the size of the page (including header)
+   * @param firstRowIndex
+   *          the index of the first row in the page (within the row group)
+   */
+  public void add(long offset, int compressedPageSize, long firstRowIndex) {
+    previousOffset = offset;
+    offsets.add(offset);
+    previousPageSize = compressedPageSize;
+    compressedPageSizes.add(compressedPageSize);
+    previousRowIndex = firstRowIndex;
+    firstRowIndexes.add(firstRowIndex);
+  }
+
+  /**
+   * Builds the offset index. Used by the metadata converter to building up {@link OffsetIndex}
+   * objects read from the Parquet file.
+   *
+   * @return the newly created offset index or {@code null} if the {@link OffsetIndex} object would be empty
+   */
+  public OffsetIndex build() {
+    return build(0);
+  }
+
+  /**
+   * Builds the offset index. Used by the writers to building up {@link OffsetIndex} objects to be
+   * written to the Parquet file.
+   *
+   * @param firstPageOffset
+   *          the actual offset in the file to be used to translate all the collected offsets
+   * @return the newly created offset index or {@code null} if the {@link OffsetIndex} object would be empty
+   */
+  public OffsetIndex build(long firstPageOffset) {
+    if (compressedPageSizes.isEmpty()) {
+      return null;
+    }
+    long[] offsets = this.offsets.toLongArray();
+    if (firstPageOffset != 0) {
+      for (int i = 0, n = offsets.length; i < n; ++i) {
+        offsets[i] += firstPageOffset;
+      }
+    }
+    OffsetIndexImpl offsetIndex = new OffsetIndexImpl();
+    offsetIndex.offsets = offsets;
+    offsetIndex.compressedPageSizes = compressedPageSizes.toIntArray();
+    offsetIndex.firstRowIndexes = firstRowIndexes.toLongArray();
+
+    return offsetIndex;
+  }
+
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
new file mode 100644
index 0000000..fb3077e
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexFilter.java
@@ -0,0 +1,194 @@
+/*
+ * 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.parquet.internal.filter2.columnindex;
+
+import java.util.PrimitiveIterator;
+import java.util.Set;
+import java.util.function.Function;
+
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.FilterPredicateCompat;
+import org.apache.parquet.filter2.compat.FilterCompat.NoOpFilter;
+import org.apache.parquet.filter2.compat.FilterCompat.UnboundRecordFilterCompat;
+import org.apache.parquet.filter2.predicate.FilterPredicate.Visitor;
+import org.apache.parquet.filter2.predicate.Operators.And;
+import org.apache.parquet.filter2.predicate.Operators.Column;
+import org.apache.parquet.filter2.predicate.Operators.Eq;
+import org.apache.parquet.filter2.predicate.Operators.Gt;
+import org.apache.parquet.filter2.predicate.Operators.GtEq;
+import org.apache.parquet.filter2.predicate.Operators.LogicalNotUserDefined;
+import org.apache.parquet.filter2.predicate.Operators.Lt;
+import org.apache.parquet.filter2.predicate.Operators.LtEq;
+import org.apache.parquet.filter2.predicate.Operators.Not;
+import org.apache.parquet.filter2.predicate.Operators.NotEq;
+import org.apache.parquet.filter2.predicate.Operators.Or;
+import org.apache.parquet.filter2.predicate.Operators.UserDefined;
+import org.apache.parquet.filter2.predicate.UserDefinedPredicate;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.internal.filter2.columnindex.ColumnIndexStore.MissingOffsetIndexException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Filter implementation based on column indexes.
+ * No filtering will be applied for columns where no column index is available.
+ * Offset index is required for all the columns in the projection, therefore a {@link MissingOffsetIndexException} will
+ * be thrown from any {@code visit} methods if any of the required offset indexes is missing.
+ */
+public class ColumnIndexFilter implements Visitor<RowRanges> {
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(ColumnIndexFilter.class);
+  private final ColumnIndexStore columnIndexStore;
+  private final Set<ColumnPath> columns;
+  private final long rowCount;
+  private RowRanges allRows;
+
+  /**
+   * Calculates the row ranges containing the indexes of the rows might match the specified filter.
+   *
+   * @param filter
+   *          to be used for filtering the rows
+   * @param columnIndexStore
+   *          the store for providing column/offset indexes
+   * @param paths
+   *          the paths of the columns used in the actual projection; a column not being part of the projection will be
+   *          handled as containing {@code null} values only even if the column has values written in the file
+   * @param rowCount
+   *          the total number of rows in the row-group
+   * @return the ranges of the possible matching row indexes; the returned ranges will contain all the rows if any of
+   *         the required offset index is missing
+   */
+  public static RowRanges calculateRowRanges(FilterCompat.Filter filter, ColumnIndexStore columnIndexStore,
+      Set<ColumnPath> paths, long rowCount) {
+    return filter.accept(new FilterCompat.Visitor<RowRanges>() {
+      @Override
+      public RowRanges visit(FilterPredicateCompat filterPredicateCompat) {
+        try {
+          return filterPredicateCompat.getFilterPredicate()
+              .accept(new ColumnIndexFilter(columnIndexStore, paths, rowCount));
+        } catch (MissingOffsetIndexException e) {
+          LOGGER.warn("Unable to do filtering", e);
+          return RowRanges.createSingle(rowCount);
+        }
+      }
+
+      @Override
+      public RowRanges visit(UnboundRecordFilterCompat unboundRecordFilterCompat) {
+        return RowRanges.createSingle(rowCount);
+      }
+
+      @Override
+      public RowRanges visit(NoOpFilter noOpFilter) {
+        return RowRanges.createSingle(rowCount);
+      }
+    });
+  }
+
+  private ColumnIndexFilter(ColumnIndexStore columnIndexStore, Set<ColumnPath> paths, long rowCount) {
+    this.columnIndexStore = columnIndexStore;
+    this.columns = paths;
+    this.rowCount = rowCount;
+  }
+
+  private RowRanges allRows() {
+    if (allRows == null) {
+      allRows = RowRanges.createSingle(rowCount);
+    }
+    return allRows;
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(Eq<T> eq) {
+    return applyPredicate(eq.getColumn(), ci -> ci.visit(eq), eq.getValue() == null ? allRows() : RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(NotEq<T> notEq) {
+    return applyPredicate(notEq.getColumn(), ci -> ci.visit(notEq),
+        notEq.getValue() == null ? RowRanges.EMPTY : allRows());
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(Lt<T> lt) {
+    return applyPredicate(lt.getColumn(), ci -> ci.visit(lt), RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(LtEq<T> ltEq) {
+    return applyPredicate(ltEq.getColumn(), ci -> ci.visit(ltEq), RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(Gt<T> gt) {
+    return applyPredicate(gt.getColumn(), ci -> ci.visit(gt), RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>> RowRanges visit(GtEq<T> gtEq) {
+    return applyPredicate(gtEq.getColumn(), ci -> ci.visit(gtEq), RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> RowRanges visit(UserDefined<T, U> udp) {
+    return applyPredicate(udp.getColumn(), ci -> ci.visit(udp),
+        udp.getUserDefinedPredicate().keep(null) ? allRows() : RowRanges.EMPTY);
+  }
+
+  @Override
+  public <T extends Comparable<T>, U extends UserDefinedPredicate<T>> RowRanges visit(
+      LogicalNotUserDefined<T, U> udp) {
+    return applyPredicate(udp.getUserDefined().getColumn(), ci -> ci.visit(udp),
+        udp.getUserDefined().getUserDefinedPredicate().keep(null) ? RowRanges.EMPTY : allRows());
+  }
+
+  private RowRanges applyPredicate(Column<?> column, Function<ColumnIndex, PrimitiveIterator.OfInt> func,
+      RowRanges rangesForMissingColumns) {
+    ColumnPath columnPath = column.getColumnPath();
+    if (!columns.contains(columnPath)) {
+      return rangesForMissingColumns;
+    }
+
+    OffsetIndex oi = columnIndexStore.getOffsetIndex(columnPath);
+    ColumnIndex ci = columnIndexStore.getColumnIndex(columnPath);
+    if (ci == null) {
+      LOGGER.warn("No column index for column {} is available; Unable to filter on this column", columnPath);
+      return allRows();
+    }
+
+    return RowRanges.create(rowCount, func.apply(ci), oi);
+  }
+
+  @Override
+  public RowRanges visit(And and) {
+    return RowRanges.intersection(and.getLeft().accept(this), and.getRight().accept(this));
+  }
+
+  @Override
+  public RowRanges visit(Or or) {
+    return RowRanges.union(or.getLeft().accept(this), or.getRight().accept(this));
+  }
+
+  @Override
+  public RowRanges visit(Not not) {
+    throw new IllegalArgumentException(
+        "Predicates containing a NOT must be run through LogicalInverseRewriter. " + not);
+  }
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexStore.java b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexStore.java
new file mode 100644
index 0000000..c82861a
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/ColumnIndexStore.java
@@ -0,0 +1,55 @@
+/*
+ * 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.parquet.internal.filter2.columnindex;
+
+import org.apache.parquet.ParquetRuntimeException;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+
+/**
+ * Provides the {@link ColumnIndex} and {@link OffsetIndex} objects for a row-group.
+ */
+public interface ColumnIndexStore {
+
+  /**
+   * Exception thrown in case of an offset index is missing for any of the columns.
+   */
+  public static class MissingOffsetIndexException extends ParquetRuntimeException {
+    public MissingOffsetIndexException(ColumnPath path) {
+      super("No offset index for column " + path.toDotString() + " is available; Unable to do filtering");
+    }
+  }
+
+  /**
+   * @param column
+   *          the path of the column
+   * @return the column index for the column-chunk in the row-group or {@code null} if no column index is available
+   */
+  ColumnIndex getColumnIndex(ColumnPath column);
+
+  /**
+   * @param column
+   *          the path of the column
+   * @return the offset index for the column-chunk in the row-group
+   * @throws MissingOffsetIndexException
+   *           if the related offset index is missing
+   */
+  OffsetIndex getOffsetIndex(ColumnPath column) throws MissingOffsetIndexException;
+}
diff --git a/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java
new file mode 100644
index 0000000..7753507
--- /dev/null
+++ b/parquet-column/src/main/java/org/apache/parquet/internal/filter2/columnindex/RowRanges.java
@@ -0,0 +1,288 @@
+/*
+ * 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.parquet.internal.filter2.columnindex;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+import java.util.PrimitiveIterator;
+import java.util.Set;
+
+import org.apache.parquet.filter2.compat.FilterCompat.Filter;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+
+/**
+ * Class representing row ranges in a row-group. These row ranges are calculated as a result of the column index based
+ * filtering. To be used iterate over the matching row indexes to be read from a row-group, retrieve the count of the
+ * matching rows or check overlapping of a row index range.
+ *
+ * @see ColumnIndexFilter#calculateRowRanges(Filter, ColumnIndexStore, Set, long)
+ */
+public class RowRanges {
+  private static class Range {
+
+    // Returns the union of the two ranges or null if there are elements between them.
+    private static Range union(Range left, Range right) {
+      if (left.from <= right.from) {
+        if (left.to + 1 >= right.from) {
+          return new Range(left.from, Math.max(left.to, right.to));
+        }
+      } else if (right.to + 1 >= left.from) {
+        return new Range(right.from, Math.max(left.to, right.to));
+      }
+      return null;
+    }
+
+    // Returns the intersection of the two ranges of null if they are not overlapped.
+    private static Range intersection(Range left, Range right) {
+      if (left.from <= right.from) {
+        if (left.to >= right.from) {
+          return new Range(right.from, Math.min(left.to, right.to));
+        }
+      } else if (right.to >= left.from) {
+        return new Range(left.from, Math.min(left.to, right.to));
+      }
+      return null;
+    }
+
+    final long from;
+    final long to;
+
+    // Creates a range of [from, to] (from and to are inclusive; empty ranges are not valid)
+    Range(long from, long to) {
+      assert from <= to;
+      this.from = from;
+      this.to = to;
+    }
+
+    long count() {
+      return to - from + 1;
+    }
+
+    boolean isBefore(Range other) {
+      return to < other.from;
+    }
+
+    boolean isAfter(Range other) {
+      return from > other.to;
+    }
+
+    @Override
+    public String toString() {
+      return "[" + from + ", " + to + ']';
+    }
+  }
+
+  static final RowRanges EMPTY = new RowRanges();
+
+  /*
+   * Creates a new RowRanges object with the single range [0, rowCount - 1].
+   */
+  static RowRanges createSingle(long rowCount) {
+    RowRanges ranges = new RowRanges();
+    ranges.add(new Range(0, rowCount - 1));
+    return ranges;
+  }
+
+  /*
+   * Creates a new RowRanges object with the following ranges.
+   * [firstRowIndex[0], lastRowIndex[0]],
+   * [firstRowIndex[1], lastRowIndex[1]],
+   * ...,
+   * [firstRowIndex[n], lastRowIndex[n]]
+   * (See OffsetIndex.getFirstRowIndex and OffsetIndex.getLastRowIndex for details.)
+   *
+   * The union of the ranges are calculated so the result ranges always contain the disjunct ranges. See union for
+   * details.
+   */
+  static RowRanges create(long rowCount, PrimitiveIterator.OfInt pageIndexes, OffsetIndex offsetIndex) {
+    RowRanges ranges = new RowRanges();
+    while (pageIndexes.hasNext()) {
+      int pageIndex = pageIndexes.nextInt();
+      ranges.add(new Range(offsetIndex.getFirstRowIndex(pageIndex), offsetIndex.getLastRowIndex(pageIndex, rowCount)));
+    }
+    return ranges;
+  }
+
+  /*
+   * Calculates the union of the two specified RowRanges object. The union of two range is calculated if there are no
+   * elements between them. Otherwise, the two disjunct ranges are stored separately.
+   * For example:
+   * [113, 241] ∪ [221, 340] = [113, 330]
+   * [113, 230] ∪ [231, 340] = [113, 340]
+   * while
+   * [113, 230] ∪ [232, 340] = [113, 230], [232, 340]
+   *
+   * The result RowRanges object will contain all the row indexes that were contained in one of the specified objects.
+   */
+  static RowRanges union(RowRanges left, RowRanges right) {
+    RowRanges result = new RowRanges();
+    Iterator<Range> it1 = left.ranges.iterator();
+    Iterator<Range> it2 = right.ranges.iterator();
+    if (it2.hasNext()) {
+      Range range2 = it2.next();
+      while (it1.hasNext()) {
+        Range range1 = it1.next();
+        if (range1.isAfter(range2)) {
+          result.add(range2);
+          range2 = range1;
+          Iterator<Range> tmp = it1;
+          it1 = it2;
+          it2 = tmp;
+        } else {
+          result.add(range1);
+        }
+      }
+      result.add(range2);
+    } else {
+      it2 = it1;
+    }
+    while (it2.hasNext()) {
+      result.add(it2.next());
+    }
+
+    return result;
+  }
+
+  /*
+   * Calculates the intersection of the two specified RowRanges object. Two ranges intersect if they have common
+   * elements otherwise the result is empty.
+   * For example:
+   * [113, 241] ∩ [221, 340] = [221, 241]
+   * while
+   * [113, 230] ∩ [231, 340] = <EMPTY>
+   *
+   * The result RowRanges object will contain all the row indexes there were contained in both of the specified objects
+   */
+  static RowRanges intersection(RowRanges left, RowRanges right) {
+    RowRanges result = new RowRanges();
+
+    int rightIndex = 0;
+    for (Range l : left.ranges) {
+      for (int i = rightIndex, n = right.ranges.size(); i < n; ++i) {
+        Range r = right.ranges.get(i);
+        if (l.isBefore(r)) {
+          break;
+        } else if (l.isAfter(r)) {
+          rightIndex = i + 1;
+          continue;
+        }
+        result.add(Range.intersection(l, r));
+      }
+    }
+
+    return result;
+  }
+
+  private final List<Range> ranges = new ArrayList<>();
+
+  private RowRanges() {
+  }
+
+  /*
+   * Adds a range to the end of the list of ranges. It maintains the disjunct ascending order(*) of the ranges by
+   * trying to union the specified range to the last ranges in the list. The specified range shall be larger(*) than
+   * the last one or might be overlapped with some of the last ones.
+   * (*) [a, b] < [c, d] if b < c
+   */
+  private void add(Range range) {
+    Range rangeToAdd = range;
+    for (int i = ranges.size() - 1; i >= 0; --i) {
+      Range last = ranges.get(i);
+      assert !last.isAfter(range);
+      Range u = Range.union(last, rangeToAdd);
+      if (u == null) {
+        break;
+      }
+      rangeToAdd = u;
+      ranges.remove(i);
+    }
+    ranges.add(rangeToAdd);
+  }
+
+  /**
+   * @return the number of rows in the ranges
+   */
+  public long rowCount() {
+    long cnt = 0;
+    for (Range range : ranges) {
+      cnt += range.count();
+    }
+    return cnt;
+  }
+
+  /**
+   * @return the ascending iterator of the row indexes contained in the ranges
+   */
+  public PrimitiveIterator.OfLong iterator() {
+    return new PrimitiveIterator.OfLong() {
+      private int currentRangeIndex = -1;
+      private Range currentRange;
+      private long next = findNext();
+
+      private long findNext() {
+        if (currentRange == null || next + 1 > currentRange.to) {
+          if (currentRangeIndex + 1 < ranges.size()) {
+            currentRange = ranges.get(++currentRangeIndex);
+            next = currentRange.from;
+          } else {
+            return -1;
+          }
+        } else {
+          ++next;
+        }
+        return next;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return next >= 0;
+      }
+
+      @Override
+      public long nextLong() {
+        long ret = next;
+        if (ret < 0) {
+          throw new NoSuchElementException();
+        }
+        next = findNext();
+        return ret;
+      }
+    };
+  }
+
+  /**
+   * @param from
+   *          the first row of the range to be checked for connection
+   * @param to
+   *          the last row of the range to be checked for connection
+   * @return {@code true} if the specified range is overlapping (have common elements) with one of the ranges
+   */
+  public boolean isOverlapping(long from, long to) {
+    return Collections.binarySearch(ranges, new Range(from, to),
+        (r1, r2) -> r1.isBefore(r2) ? -1 : r1.isAfter(r2) ? 1 : 0) >= 0;
+  }
+
+  @Override
+  public String toString() {
+    return ranges.toString();
+  }
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
index d2d78c4..35fddaf 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/impl/TestColumnReaderImpl.java
@@ -65,10 +65,10 @@ public class TestColumnReaderImpl {
     for (int i = 0; i < rows; i++) {
       columnWriterV2.write(Binary.fromString("bar" + i % 10), 0, 0);
       if ((i + 1) % 1000 == 0) {
-        columnWriterV2.writePage(i);
+        columnWriterV2.writePage();
       }
     }
-    columnWriterV2.writePage(rows);
+    columnWriterV2.writePage();
     columnWriterV2.finalizeColumnChunk();
     List<DataPage> pages = pageWriter.getPages();
     int valueCount = 0;
@@ -103,10 +103,10 @@ public class TestColumnReaderImpl {
     for (int i = 0; i < rows; i++) {
       columnWriterV2.writeNull(0, 0);
       if ((i + 1) % 1000 == 0) {
-        columnWriterV2.writePage(i);
+        columnWriterV2.writePage();
       }
     }
-    columnWriterV2.writePage(rows);
+    columnWriterV2.writePage();
     columnWriterV2.finalizeColumnChunk();
     List<DataPage> pages = pageWriter.getPages();
     int valueCount = 0;
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
index c855339..e5db38c 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/mem/TestMemColumn.java
@@ -20,12 +20,10 @@ package org.apache.parquet.column.mem;
 
 import static org.junit.Assert.assertEquals;
 
-import org.apache.parquet.column.ParquetProperties;
-import org.junit.Test;
-
 import org.apache.parquet.column.ColumnDescriptor;
 import org.apache.parquet.column.ColumnReader;
 import org.apache.parquet.column.ColumnWriter;
+import org.apache.parquet.column.ParquetProperties;
 import org.apache.parquet.column.impl.ColumnReadStoreImpl;
 import org.apache.parquet.column.impl.ColumnWriteStoreV1;
 import org.apache.parquet.column.page.mem.MemPageStore;
@@ -33,6 +31,7 @@ import org.apache.parquet.example.DummyRecordConverter;
 import org.apache.parquet.io.api.Binary;
 import org.apache.parquet.schema.MessageType;
 import org.apache.parquet.schema.MessageTypeParser;
+import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -47,6 +46,7 @@ public class TestMemColumn {
     ColumnWriteStoreV1 memColumnsStore = newColumnWriteStoreImpl(memPageStore);
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     columnWriter.write(42l, 0, 0);
+    memColumnsStore.endRecord();
     memColumnsStore.flush();
 
     ColumnReader columnReader = getColumnReader(memPageStore, path, schema);
@@ -85,6 +85,7 @@ public class TestMemColumn {
 
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     columnWriter.write(Binary.fromString("42"), 0, 0);
+    memColumnsStore.endRecord();
     memColumnsStore.flush();
 
     ColumnReader columnReader = getColumnReader(memPageStore, path, mt);
@@ -108,6 +109,7 @@ public class TestMemColumn {
     ColumnWriter columnWriter = memColumnsStore.getColumnWriter(path);
     for (int i = 0; i < 2000; i++) {
       columnWriter.write(42l, 0, 0);
+      memColumnsStore.endRecord();
     }
     memColumnsStore.flush();
 
@@ -136,12 +138,16 @@ public class TestMemColumn {
       int r = rs[i % rs.length];
       int d = ds[i % ds.length];
       LOG.debug("write i: {}", i);
+      if (i != 0 && r == 0) {
+        memColumnsStore.endRecord();
+      }
       if (d == 2) {
         columnWriter.write((long)i, r, d);
       } else {
         columnWriter.writeNull(r, d);
       }
     }
+    memColumnsStore.endRecord();
     memColumnsStore.flush();
 
     ColumnReader columnReader = getColumnReader(memPageStore, path, mt);
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
index be3a0f9..706b001 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/page/mem/MemPageWriter.java
@@ -57,6 +57,12 @@ public class MemPageWriter implements PageWriter {
   }
 
   @Override
+  public void writePage(BytesInput bytesInput, int valueCount, int rowCount, Statistics<?> statistics,
+      Encoding rlEncoding, Encoding dlEncoding, Encoding valuesEncoding) throws IOException {
+    writePage(bytesInput, valueCount, statistics, rlEncoding, dlEncoding, valuesEncoding);
+  }
+
+  @Override
   public void writePageV2(int rowCount, int nullCount, int valueCount,
       BytesInput repetitionLevels, BytesInput definitionLevels,
       Encoding dataEncoding, BytesInput data, Statistics<?> statistics) throws IOException {
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
index 867af28..3ca3d08 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/bitpacking/TestBitPackingColumn.java
@@ -183,6 +183,22 @@ public class TestBitPackingColumn {
       }
       LOG.debug("result: {}", TestBitPacking.toString(result));
       assertArrayEquals(type + " result: " + TestBitPacking.toString(result), vals, result);
+
+      // Test skipping
+      r.initFromPage(vals.length, ByteBufferInputStream.wrap(ByteBuffer.wrap(bytes)));
+      for (int i = 0; i < vals.length; i += 2) {
+        assertEquals(vals[i], r.readInteger());
+        r.skip();
+      }
+
+      // Test n-skipping
+      r.initFromPage(vals.length, ByteBufferInputStream.wrap(ByteBuffer.wrap(bytes)));
+      int skipCount;
+      for (int i = 0; i < vals.length; i += skipCount + 1) {
+        skipCount = (vals.length - i) / 2;
+        assertEquals(vals[i], r.readInteger());
+        r.skip(skipCount);
+      }
     }
   }
 
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForIntegerTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForIntegerTest.java
index df99e3c..c69e0ff 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForIntegerTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForIntegerTest.java
@@ -213,6 +213,23 @@ public class DeltaBinaryPackingValuesWriterForIntegerTest {
   }
 
   @Test
+  public void shouldSkipN() throws IOException {
+    int[] data = new int[5 * blockSize + 1];
+    for (int i = 0; i < data.length; i++) {
+      data[i] = i * 32;
+    }
+    writeData(data);
+    reader = new DeltaBinaryPackingValuesReader();
+    reader.initFromPage(100, writer.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < data.length; i += skipCount + 1) {
+      skipCount = (data.length - i) / 2;
+      assertEquals(i * 32, reader.readInteger());
+      reader.skip(skipCount);
+    }
+  }
+
+  @Test
   public void shouldReset() throws IOException {
     shouldReadWriteWhenDataIsNotAlignedWithBlock();
     int[] data = new int[5 * blockSize];
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForLongTest.java b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForLongTest.java
index 65ac819..ca12bbd 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForLongTest.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/delta/DeltaBinaryPackingValuesWriterForLongTest.java
@@ -212,6 +212,23 @@ public class DeltaBinaryPackingValuesWriterForLongTest {
   }
 
   @Test
+  public void shouldSkipN() throws IOException {
+    long[] data = new long[5 * blockSize + 1];
+    for (int i = 0; i < data.length; i++) {
+      data[i] = i * 32;
+    }
+    writeData(data);
+    reader = new DeltaBinaryPackingValuesReader();
+    reader.initFromPage(100, writer.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < data.length; i += skipCount + 1) {
+      skipCount = (data.length - i) / 2;
+      assertEquals(i * 32, reader.readLong());
+      reader.skip(skipCount);
+    }
+  }
+
+  @Test
   public void shouldReset() throws IOException {
     shouldReadWriteWhenDataIsNotAlignedWithBlock();
     long[] data = new long[5 * blockSize];
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
index d214a88..6c97430 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltalengthbytearray/TestDeltaLengthByteArray.java
@@ -65,6 +65,30 @@ public class TestDeltaLengthByteArray {
   }
 
   @Test
+  public void testSkipWithRandomStrings() throws IOException {
+    DeltaLengthByteArrayValuesWriter writer = getDeltaLengthByteArrayValuesWriter();
+    DeltaLengthByteArrayValuesReader reader = new DeltaLengthByteArrayValuesReader();
+
+    String[] values = Utils.getRandomStringSamples(1000, 32);
+    Utils.writeData(writer, values);
+
+    reader.initFromPage(values.length, writer.getBytes().toInputStream());
+    for (int i = 0; i < values.length; i += 2) {
+      Assert.assertEquals(Binary.fromString(values[i]), reader.readBytes());
+      reader.skip();
+    }
+
+    reader = new DeltaLengthByteArrayValuesReader();
+    reader.initFromPage(values.length, writer.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < values.length; i += skipCount + 1) {
+      skipCount = (values.length - i) / 2;
+      Assert.assertEquals(Binary.fromString(values[i]), reader.readBytes());
+      reader.skip(skipCount);
+    }
+  }
+
+  @Test
   public void testLengths() throws IOException {
     DeltaLengthByteArrayValuesWriter writer = getDeltaLengthByteArrayValuesWriter();
     ValuesReader reader = new DeltaBinaryPackingValuesReader();
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
index c13a3a2..a5a22a8 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/deltastrings/TestDeltaByteArray.java
@@ -59,6 +59,13 @@ public class TestDeltaByteArray {
   }
 
   @Test
+  public void testRandomStringsWithSkipN() throws Exception {
+    DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
+    DeltaByteArrayReader reader = new DeltaByteArrayReader();
+    assertReadWriteWithSkipN(writer, reader, randvalues);
+  }
+
+  @Test
   public void testLengths() throws IOException {
     DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
     ValuesReader reader = new DeltaBinaryPackingValuesReader();
@@ -99,6 +106,18 @@ public class TestDeltaByteArray {
     }
   }
 
+  private void assertReadWriteWithSkipN(DeltaByteArrayWriter writer, DeltaByteArrayReader reader, String[] vals) throws Exception {
+    Utils.writeData(writer, vals);
+
+    reader.initFromPage(vals.length, writer.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < vals.length; i += skipCount + 1) {
+      skipCount = (vals.length - i) / 2;
+      Assert.assertEquals(Binary.fromString(vals[i]), reader.readBytes());
+      reader.skip(skipCount);
+    }
+  }
+
   @Test
   public void testWriterReset() throws Exception {
     DeltaByteArrayWriter writer = new DeltaByteArrayWriter(64 * 1024, 64 * 1024, new DirectByteBufferAllocator());
diff --git a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
index ef2b721..ba3f903 100644
--- a/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
+++ b/parquet-column/src/test/java/org/apache/parquet/column/values/dictionary/TestDictionary.java
@@ -100,6 +100,47 @@ public class TestDictionary {
   }
 
   @Test
+  public void testSkipInBinaryDictionary() throws Exception {
+    ValuesWriter cw = newPlainBinaryDictionaryValuesWriter(1000, 10000);
+    writeRepeated(100, cw, "a");
+    writeDistinct(100, cw, "b");
+    assertEquals(PLAIN_DICTIONARY, cw.getEncoding());
+
+    // Test skip and skip-n with dictionary encoding
+    ByteBufferInputStream stream = cw.getBytes().toInputStream();
+    DictionaryValuesReader cr = initDicReader(cw, BINARY);
+    cr.initFromPage(200, stream);
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals(Binary.fromString("a" + i % 10), cr.readBytes());
+      cr.skip();
+    }
+    int skipCount;
+    for (int i = 0; i < 100; i += skipCount + 1) {
+      skipCount = (100 - i) / 2;
+      assertEquals(Binary.fromString("b" + i), cr.readBytes());
+      cr.skip(skipCount);
+    }
+
+    // Ensure fallback
+    writeDistinct(1000, cw, "c");
+    assertEquals(PLAIN, cw.getEncoding());
+
+    // Test skip and skip-n with plain encoding (after fallback)
+    ValuesReader plainReader = new BinaryPlainValuesReader();
+    plainReader.initFromPage(1200, cw.getBytes().toInputStream());
+    plainReader.skip(200);
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals("c" + i, plainReader.readBytes().toStringUsingUTF8());
+      plainReader.skip();
+    }
+    for (int i = 100; i < 1000; i += skipCount + 1) {
+      skipCount = (1000 - i) / 2;
+      assertEquals(Binary.fromString("c" + i), plainReader.readBytes());
+      plainReader.skip(skipCount);
+    }
+  }
+
+  @Test
   public void testBinaryDictionaryFallBack() throws IOException {
     int slabSize = 100;
     int maxDictionaryByteSize = 50;
@@ -234,6 +275,22 @@ public class TestDictionary {
     for (long i = 0; i < 100; i++) {
       assertEquals(i, reader.readLong());
     }
+
+    // Test skip with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals(i, reader.readLong());
+      reader.skip();
+    }
+
+    // Test skip-n with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < 100; i += skipCount + 1) {
+      skipCount = (100 - i) / 2;
+      assertEquals(i, reader.readLong());
+      reader.skip(skipCount);
+    }
   }
 
   @Test
@@ -305,6 +362,22 @@ public class TestDictionary {
     for (double i = 0; i < 100; i++) {
       assertEquals(i, reader.readDouble(), 0.00001);
     }
+
+    // Test skip with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals(i, reader.readDouble(), 0.0);
+      reader.skip();
+    }
+
+    // Test skip-n with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < 100; i += skipCount + 1) {
+      skipCount = (100 - i) / 2;
+      assertEquals(i, reader.readDouble(), 0.0);
+      reader.skip(skipCount);
+    }
   }
 
   @Test
@@ -376,6 +449,22 @@ public class TestDictionary {
     for (int i = 0; i < 100; i++) {
       assertEquals(i, reader.readInteger());
     }
+
+    // Test skip with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals(i, reader.readInteger());
+      reader.skip();
+    }
+
+    // Test skip-n with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < 100; i += skipCount + 1) {
+      skipCount = (100 - i) / 2;
+      assertEquals(i, reader.readInteger());
+      reader.skip(skipCount);
+    }
   }
 
   @Test
@@ -447,6 +536,22 @@ public class TestDictionary {
     for (float i = 0; i < 100; i++) {
       assertEquals(i, reader.readFloat(), 0.00001);
     }
+
+    // Test skip with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    for (int i = 0; i < 100; i += 2) {
+      assertEquals(i, reader.readFloat(), 0.0f);
+      reader.skip();
+    }
+
+    // Test skip-n with plain encoding
+    reader.initFromPage(100, cw.getBytes().toInputStream());
+    int skipCount;
+    for (int i = 0; i < 100; i += skipCount + 1) {
+      skipCount = (100 - i) / 2;
+      assertEquals(i, reader.readFloat(), 0.0f);
+      reader.skip(skipCount);
+    }
   }
 
   @Test
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBinaryTruncator.java b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBinaryTruncator.java
new file mode 100644
index 0000000..c3e3d85
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBinaryTruncator.java
@@ -0,0 +1,285 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static org.apache.parquet.schema.OriginalType.BSON;
+import static org.apache.parquet.schema.OriginalType.DECIMAL;
+import static org.apache.parquet.schema.OriginalType.ENUM;
+import static org.apache.parquet.schema.OriginalType.INTERVAL;
+import static org.apache.parquet.schema.OriginalType.JSON;
+import static org.apache.parquet.schema.OriginalType.UTF8;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT96;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertSame;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.charset.CharacterCodingException;
+import java.nio.charset.CharsetDecoder;
+import java.nio.charset.CodingErrorAction;
+import java.nio.charset.StandardCharsets;
+import java.util.Comparator;
+import java.util.Random;
+
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveStringifier;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tests for {@link BinaryTruncator}
+ */
+public class TestBinaryTruncator {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestBinaryTruncator.class);
+  private static final PrimitiveStringifier HEXA_STRINGIFIER = Types.required(BINARY)
+      .named("dummy_type").stringifier();
+  private static final Random RANDOM = new Random(42);
+  private static final CharsetDecoder UTF8_DECODER = StandardCharsets.UTF_8.newDecoder();
+  static {
+    UTF8_DECODER.onMalformedInput(CodingErrorAction.REPORT);
+    UTF8_DECODER.onUnmappableCharacter(CodingErrorAction.REPORT);
+  }
+
+  // The maximum values in UTF-8 for the 1, 2, 3 and 4 bytes representations
+  private static final String UTF8_1BYTE_MAX_CHAR = "\u007F";
+  private static final String UTF8_2BYTES_MAX_CHAR = "\u07FF";
+  private static final String UTF8_3BYTES_MAX_CHAR = "\uFFFF";
+  private static final String UTF8_4BYTES_MAX_CHAR = "\uDBFF\uDFFF";
+
+  @Test
+  public void testNonStringTruncate() {
+    BinaryTruncator truncator = BinaryTruncator
+        .getTruncator(Types.required(BINARY).as(DECIMAL).precision(10).scale(2).named("test_binary_decimal"));
+    assertEquals(binary(0xFF, 0xFE, 0xFD, 0xFC, 0xFB, 0xFA),
+        truncator.truncateMin(binary(0xFF, 0xFE, 0xFD, 0xFC, 0xFB, 0xFA), 2));
+    assertEquals(binary(0x01, 0x02, 0x03, 0x04, 0x05, 0x06),
+        truncator.truncateMax(binary(0x01, 0x02, 0x03, 0x04, 0x05, 0x06), 2));
+  }
+
+  @Test
+  public void testContractNonStringTypes() {
+    testTruncator(
+        Types.required(FIXED_LEN_BYTE_ARRAY).length(8).as(DECIMAL).precision(18).scale(4).named("test_fixed_decimal"),
+        false);
+    testTruncator(Types.required(FIXED_LEN_BYTE_ARRAY).length(12).as(INTERVAL).named("test_fixed_interval"), false);
+    testTruncator(Types.required(BINARY).as(DECIMAL).precision(10).scale(2).named("test_binary_decimal"), false);
+    testTruncator(Types.required(INT96).named("test_int96"), false);
+  }
+
+  @Test
+  public void testStringTruncate() {
+    BinaryTruncator truncator = BinaryTruncator.getTruncator(Types.required(BINARY).as(UTF8).named("test_utf8"));
+
+    // Truncate 1 byte characters
+    assertEquals(Binary.fromString("abc"), truncator.truncateMin(Binary.fromString("abcdef"), 3));
+    assertEquals(Binary.fromString("abd"), truncator.truncateMax(Binary.fromString("abcdef"), 3));
+
+    // Truncate 1-2 bytes characters; the target length is "inside" a UTF-8 character
+    assertEquals(Binary.fromString("árvízt"), truncator.truncateMin(Binary.fromString("árvíztűrő"), 9));
+    assertEquals(Binary.fromString("árvízu"), truncator.truncateMax(Binary.fromString("árvíztűrő"), 9));
+
+    // Truncate highest UTF-8 values -> unable to increment
+    assertEquals(
+        Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR),
+        truncator.truncateMin(Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR
+                + UTF8_3BYTES_MAX_CHAR
+                + UTF8_4BYTES_MAX_CHAR),
+            5));
+    assertEquals(
+        Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR
+                + UTF8_3BYTES_MAX_CHAR
+                + UTF8_4BYTES_MAX_CHAR),
+        truncator.truncateMax(Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR
+                + UTF8_3BYTES_MAX_CHAR
+                + UTF8_4BYTES_MAX_CHAR),
+            5));
+
+    // Truncate highest UTF-8 values at the end -> increment the first possible character
+    assertEquals(
+        Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR
+                + "b"
+                + UTF8_3BYTES_MAX_CHAR),
+        truncator.truncateMax(Binary.fromString(
+            UTF8_1BYTE_MAX_CHAR
+                + UTF8_2BYTES_MAX_CHAR
+                + "a"
+                + UTF8_3BYTES_MAX_CHAR
+                + UTF8_4BYTES_MAX_CHAR),
+            10));
+
+    // Truncate invalid UTF-8 values -> truncate without validity check
+    assertEquals(binary(0xFF, 0xFE, 0xFD), truncator.truncateMin(binary(0xFF, 0xFE, 0xFD, 0xFC, 0xFB, 0xFA), 3));
+    assertEquals(binary(0xFF, 0xFE, 0xFE), truncator.truncateMax(binary(0xFF, 0xFE, 0xFD, 0xFC, 0xFB, 0xFA), 3));
+    assertEquals(binary(0xFF, 0xFE, 0xFE, 0x00, 0x00), truncator.truncateMax(binary(0xFF, 0xFE, 0xFD, 0xFF, 0xFF, 0xFF), 5));
+  }
+
+  @Test
+  public void testContractStringTypes() {
+    testTruncator(Types.required(BINARY).named("test_binary"), true);
+    testTruncator(Types.required(BINARY).as(UTF8).named("test_utf8"), true);
+    testTruncator(Types.required(BINARY).as(ENUM).named("test_enum"), true);
+    testTruncator(Types.required(BINARY).as(JSON).named("test_json"), true);
+    testTruncator(Types.required(BINARY).as(BSON).named("test_bson"), true);
+    testTruncator(Types.required(FIXED_LEN_BYTE_ARRAY).length(5).named("test_fixed"), true);
+  }
+
+  private void testTruncator(PrimitiveType type, boolean strict) {
+    BinaryTruncator truncator = BinaryTruncator.getTruncator(type);
+    Comparator<Binary> comparator = type.comparator();
+
+    checkContract(truncator, comparator, Binary.fromString("aaaaaaaaaa"), strict, strict);
+    checkContract(truncator, comparator, Binary.fromString("árvíztűrő tükörfúrógép"), strict, strict);
+    checkContract(truncator, comparator, Binary.fromString("aaaaaaaaaa" + UTF8_3BYTES_MAX_CHAR), strict, strict);
+    checkContract(truncator, comparator, Binary.fromString("a" + UTF8_3BYTES_MAX_CHAR + UTF8_1BYTE_MAX_CHAR), strict,
+        strict);
+
+    checkContract(truncator, comparator,
+        Binary.fromConstantByteArray(new byte[] { (byte) 0xFE, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, (byte) 0xFF }), strict,
+        strict);
+
+    // Edge case: zero length -> unable to truncate
+    checkContract(truncator, comparator, Binary.fromString(""), false, false);
+    // Edge case: containing only UTF-8 max characters -> unable to truncate for max
+    checkContract(truncator, comparator, Binary.fromString(
+        UTF8_1BYTE_MAX_CHAR +
+            UTF8_4BYTES_MAX_CHAR +
+            UTF8_3BYTES_MAX_CHAR +
+            UTF8_4BYTES_MAX_CHAR +
+            UTF8_2BYTES_MAX_CHAR +
+            UTF8_3BYTES_MAX_CHAR +
+            UTF8_3BYTES_MAX_CHAR +
+            UTF8_1BYTE_MAX_CHAR +
+            UTF8_2BYTES_MAX_CHAR +
+            UTF8_3BYTES_MAX_CHAR +
+            UTF8_4BYTES_MAX_CHAR),
+        strict, false);
+    // Edge case: non-UTF-8; max bytes -> unable to truncate for max
+    checkContract(
+        truncator, comparator,
+        binary(0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF, 0xFF),
+        strict, false);
+  }
+
+  // Checks the contract of truncator
+  // strict means actual truncation is required and the truncated value is a valid UTF-8 string
+  private void checkContract(BinaryTruncator truncator, Comparator<Binary> comparator, Binary value, boolean strictMin,
+      boolean strictMax) {
+    int length = value.length();
+
+    // Edge cases: returning the original value if no truncation is required
+    assertSame(value, truncator.truncateMin(value, length));
+    assertSame(value, truncator.truncateMax(value, length));
+    assertSame(value, truncator.truncateMin(value, random(length + 1, length * 2 + 1)));
+    assertSame(value, truncator.truncateMax(value, random(length + 1, length * 2 + 1)));
+
+    if (length > 1) {
+      checkMinContract(truncator, comparator, value, length - 1, strictMin);
+      checkMaxContract(truncator, comparator, value, length - 1, strictMax);
+      checkMinContract(truncator, comparator, value, random(1, length - 1), strictMin);
+      checkMaxContract(truncator, comparator, value, random(1, length - 1), strictMax);
+    }
+
+    // Edge case: possible to truncate min value to 0 length if original value is not empty
+    checkMinContract(truncator, comparator, value, 0, strictMin);
+    // Edge case: impossible to truncate max value to 0 length -> returning the original value
+    assertSame(value, truncator.truncateMax(value, 0));
+  }
+
+  private void checkMinContract(BinaryTruncator truncator, Comparator<Binary> comparator, Binary value, int length,
+      boolean strict) {
+    Binary truncated = truncator.truncateMin(value, length);
+    LOG.debug("\"{}\" --truncMin({})--> \"{}\" [{}]", value.toStringUsingUTF8(), length, truncated.toStringUsingUTF8(),
+        HEXA_STRINGIFIER.stringify(truncated));
+    assertTrue("truncatedMin(value) should be <= than value", comparator.compare(truncated, value) <= 0);
+    assertFalse("length of truncateMin(value) should not be > than the length of value",
+        truncated.length() > value.length());
+    if (isValidUtf8(value)) {
+      checkValidUtf8(truncated);
+    }
+    if (strict) {
+      assertTrue("length of truncateMin(value) ahould be < than the length of value",
+          truncated.length() < value.length());
+    }
+  }
+
+  private void checkMaxContract(BinaryTruncator truncator, Comparator<Binary> comparator, Binary value, int length,
+      boolean strict) {
+    Binary truncated = truncator.truncateMax(value, length);
+    LOG.debug("\"{}\" --truncMax({})--> \"{}\" [{}]", value.toStringUsingUTF8(), length, truncated.toStringUsingUTF8(),
+        HEXA_STRINGIFIER.stringify(truncated));
+    assertTrue("truncatedMax(value) should be >= than value", comparator.compare(truncated, value) >= 0);
+    assertFalse("length of truncateMax(value) should not be > than the length of value",
+        truncated.length() > value.length());
+    if (isValidUtf8(value)) {
+      checkValidUtf8(truncated);
+    }
+    if (strict) {
+      assertTrue("length of truncateMax(value) ahould be < than the length of value",
+          truncated.length() < value.length());
+    }
+  }
+
+  private static boolean isValidUtf8(Binary binary) {
+    try {
+      UTF8_DECODER.decode(binary.toByteBuffer());
+      return true;
+    } catch (CharacterCodingException e) {
+      return false;
+    }
+  }
+
+  private static void checkValidUtf8(Binary binary) {
+    try {
+      UTF8_DECODER.decode(binary.toByteBuffer());
+    } catch (CharacterCodingException e) {
+      throw new AssertionError("Truncated value should be a valid UTF-8 string", e);
+    }
+  }
+
+  private static int random(int min, int max) {
+    return RANDOM.nextInt(max - min + 1) + min;
+  }
+
+  private static Binary binary(int... unsignedBytes) {
+    byte[] byteArray = new byte[unsignedBytes.length];
+    for (int i = 0, n = byteArray.length; i < n; ++i) {
+      int b = unsignedBytes[i];
+      assert (0xFFFFFF00 & b) == 0;
+      byteArray[i] = (byte) b;
+    }
+    return Binary.fromConstantByteArray(byteArray);
+  }
+
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBoundaryOrder.java b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBoundaryOrder.java
new file mode 100644
index 0000000..3d2a924
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestBoundaryOrder.java
@@ -0,0 +1,487 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.PrimitiveIterator;
+import java.util.Random;
+import java.util.function.Function;
+import java.util.stream.IntStream;
+
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder.ColumnIndexBase;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName;
+import org.apache.parquet.schema.Types;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+/**
+ * Tests the operator implementations in {@link BoundaryOrder}.
+ */
+public class TestBoundaryOrder {
+  private static class SpyValueComparatorBuilder extends ColumnIndexBase<Integer> {
+    class SpyValueComparator extends ValueComparator {
+      private final ColumnIndexBase<?>.ValueComparator delegate;
+      private int compareCount;
+
+      SpyValueComparator(ColumnIndexBase<?>.ValueComparator delegate) {
+        this.delegate = delegate;
+      }
+
+      int getCompareCount() {
+        return compareCount;
+      }
+
+      @Override
+      int arrayLength() {
+        return delegate.arrayLength();
+      }
+
+      @Override
+      int translate(int arrayIndex) {
+        return delegate.translate(arrayIndex);
+      }
+
+      @Override
+      int compareValueToMin(int arrayIndex) {
+        ++compareCount;
+        return delegate.compareValueToMin(arrayIndex);
+      }
+
+      @Override
+      int compareValueToMax(int arrayIndex) {
+        ++compareCount;
+        return delegate.compareValueToMax(arrayIndex);
+      }
+    }
+
+    private SpyValueComparatorBuilder() {
+      super(TYPE);
+    }
+
+    SpyValueComparator build(ColumnIndexBase<?>.ValueComparator comparator) {
+      return new SpyValueComparator(comparator);
+    }
+
+    @Override
+    ByteBuffer getMinValueAsBytes(int arrayIndex) {
+      throw new Error("Shall never be invoked");
+    }
+
+    @Override
+    ByteBuffer getMaxValueAsBytes(int arrayIndex) {
+      throw new Error("Shall never be invoked");
+    }
+
+    @Override
+    String getMinValueAsString(int arrayIndex) {
+      throw new Error("Shall never be invoked");
+    }
+
+    @Override
+    String getMaxValueAsString(int arrayIndex) {
+      throw new Error("Shall never be invoked");
+    }
+
+    @Override
+    <T extends Comparable<T>> org.apache.parquet.filter2.predicate.Statistics<T> createStats(int arrayIndex) {
+      throw new Error("Shall never be invoked");
+    }
+
+    @Override
+    ColumnIndexBase<Integer>.ValueComparator createValueComparator(Object value) {
+      throw new Error("Shall never be invoked");
+    }
+  }
+
+  private static class ExecStats {
+    private long linearTime;
+    private long binaryTime;
+    private int linearCompareCount;
+    private int binaryCompareCount;
+    private int execCount;
+
+    IntList measureLinear(Function<ColumnIndexBase<?>.ValueComparator, PrimitiveIterator.OfInt> op,
+        ColumnIndexBase<?>.ValueComparator comparator) {
+      IntList list = new IntArrayList(comparator.arrayLength());
+      SpyValueComparatorBuilder.SpyValueComparator spyComparator = SPY_COMPARATOR_BUILDER.build(comparator);
+      long start = System.nanoTime();
+      op.apply(spyComparator).forEachRemaining((int value) -> list.add(value));
+      linearTime = System.nanoTime() - start;
+      linearCompareCount += spyComparator.getCompareCount();
+      return list;
+    }
+
+    IntList measureBinary(Function<ColumnIndexBase<?>.ValueComparator, PrimitiveIterator.OfInt> op,
+        ColumnIndexBase<?>.ValueComparator comparator) {
+      IntList list = new IntArrayList(comparator.arrayLength());
+      SpyValueComparatorBuilder.SpyValueComparator spyComparator = SPY_COMPARATOR_BUILDER.build(comparator);
+      long start = System.nanoTime();
+      op.apply(spyComparator).forEachRemaining((int value) -> list.add(value));
+      binaryTime = System.nanoTime() - start;
+      binaryCompareCount += spyComparator.getCompareCount();
+      return list;
+    }
+
+    void add(ExecStats stats) {
+      linearTime += stats.linearTime;
+      linearCompareCount += stats.linearCompareCount;
+      binaryTime += stats.binaryTime;
+      binaryCompareCount += stats.binaryCompareCount;
+      ++execCount;
+    }
+
+    @Override
+    public String toString() {
+      double linearMs = linearTime / 1_000_000.0;
+      double binaryMs = binaryTime / 1_000_000.0;
+      return String.format(
+          "Linear search: %.2fms (avg: %.6fms); number of compares: %d (avg: %d) [100.00%%]%n"
+              + "Binary search: %.2fms (avg: %.6fms); number of compares: %d (avg: %d) [%.2f%%]",
+          linearMs, linearMs / execCount, linearCompareCount, linearCompareCount / execCount,
+          binaryMs, binaryMs / execCount, binaryCompareCount, binaryCompareCount / execCount,
+          100.0 * binaryCompareCount / linearCompareCount);
+    }
+  }
+
+  private static final Logger LOGGER = LoggerFactory.getLogger(TestBoundaryOrder.class);
+  private static final PrimitiveType TYPE = Types.required(PrimitiveTypeName.INT32).named("test_int32");
+  private static final int FROM = -15;
+  private static final int TO = 15;
+  private static final ColumnIndexBase<?> ASCENDING;
+  private static final ColumnIndexBase<?> DESCENDING;
+  private static final int SINGLE_FROM = -1;
+  private static final int SINGLE_TO = 1;
+  private static final ColumnIndexBase<?> SINGLE;
+  private static final Random RANDOM = new Random(42);
+  private static final int RAND_FROM = -2000;
+  private static final int RAND_TO = 2000;
+  private static final int RAND_COUNT = 2000;
+  private static final ColumnIndexBase<?> RAND_ASCENDING;
+  private static final ColumnIndexBase<?> RAND_DESCENDING;
+  private static final SpyValueComparatorBuilder SPY_COMPARATOR_BUILDER = new SpyValueComparatorBuilder();
+  static {
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(TYPE, Integer.MAX_VALUE);
+    builder.add(stats(FROM, -12));
+    builder.add(stats(-10, -8));
+    builder.add(stats(-8, -4));
+    builder.add(stats(-6, -4));
+    builder.add(stats(-6, -3));
+    builder.add(stats(-6, -3));
+    builder.add(stats(-6, -3));
+    builder.add(stats(0, 3));
+    builder.add(stats(3, 5));
+    builder.add(stats(3, 5));
+    builder.add(stats(5, 8));
+    builder.add(stats(10, TO));
+    ASCENDING = (ColumnIndexBase<?>) builder.build();
+
+    builder = ColumnIndexBuilder.getBuilder(TYPE, Integer.MAX_VALUE);
+    builder.add(stats(10, TO));
+    builder.add(stats(5, 8));
+    builder.add(stats(3, 5));
+    builder.add(stats(3, 5));
+    builder.add(stats(0, 3));
+    builder.add(stats(-6, -3));
+    builder.add(stats(-6, -3));
+    builder.add(stats(-6, -3));
+    builder.add(stats(-6, -4));
+    builder.add(stats(-8, -4));
+    builder.add(stats(-10, -8));
+    builder.add(stats(FROM, -12));
+    DESCENDING = (ColumnIndexBase<?>) builder.build();
+
+    builder = ColumnIndexBuilder.getBuilder(TYPE, Integer.MAX_VALUE);
+    builder.add(stats(SINGLE_FROM, SINGLE_TO));
+    SINGLE = (ColumnIndexBase<?>) builder.build();
+
+    builder = ColumnIndexBuilder.getBuilder(TYPE, Integer.MAX_VALUE);
+    for (PrimitiveIterator.OfInt it = IntStream.generate(() -> RANDOM.nextInt(RAND_TO - RAND_FROM + 1) + RAND_FROM)
+        .limit(RAND_COUNT * 2).sorted().iterator(); it.hasNext();) {
+      builder.add(stats(it.nextInt(), it.nextInt()));
+    }
+    RAND_ASCENDING = (ColumnIndexBase<?>) builder.build();
+
+    builder = ColumnIndexBuilder.getBuilder(TYPE, Integer.MAX_VALUE);
+    for (Iterator<Integer> it = IntStream.generate(() -> RANDOM.nextInt(RAND_TO - RAND_FROM + 1) + RAND_FROM)
+        .limit(RAND_COUNT * 2).mapToObj(Integer::valueOf).sorted(Collections.reverseOrder()).iterator(); it
+            .hasNext();) {
+      builder.add(stats(it.next(), it.next()));
+    }
+    RAND_DESCENDING = (ColumnIndexBase<?>) builder.build();
+  }
+
+  private static Statistics<?> stats(int min, int max) {
+    Statistics<?> stats = Statistics.createStats(TYPE);
+    stats.updateStats(min);
+    stats.updateStats(max);
+    return stats;
+  }
+
+  private static ExecStats validateOperator(String msg,
+      Function<ColumnIndexBase<?>.ValueComparator, PrimitiveIterator.OfInt> validatorOp,
+      Function<ColumnIndexBase<?>.ValueComparator, PrimitiveIterator.OfInt> actualOp,
+      ColumnIndexBase<?>.ValueComparator comparator) {
+    ExecStats stats = new ExecStats();
+
+    IntList expected = stats.measureLinear(validatorOp, comparator);
+    IntList actual = stats.measureBinary(actualOp, comparator);
+
+    Assert.assertEquals(msg, expected, actual);
+
+    return stats;
+  }
+
+  @Test
+  public void testEq() {
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.ASCENDING::eq,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.DESCENDING::eq,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = SINGLE_FROM - 1; i <= SINGLE_TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.ASCENDING::eq,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.DESCENDING::eq,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.ASCENDING::eq,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::eq,
+          BoundaryOrder.DESCENDING::eq,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed eq on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+  @Test
+  public void testGt() {
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.ASCENDING::gt,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.DESCENDING::gt,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = SINGLE_FROM - 1; i <= SINGLE_TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.ASCENDING::gt,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.DESCENDING::gt,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.ASCENDING::gt,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gt,
+          BoundaryOrder.DESCENDING::gt,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed gt on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+  @Test
+  public void testGtEq() {
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.ASCENDING::gtEq,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.DESCENDING::gtEq,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = SINGLE_FROM - 1; i <= SINGLE_TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.ASCENDING::gtEq,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.DESCENDING::gtEq,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.ASCENDING::gtEq,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::gtEq,
+          BoundaryOrder.DESCENDING::gtEq,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed gtEq on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+  @Test
+  public void testLt() {
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.ASCENDING::lt,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.DESCENDING::lt,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = SINGLE_FROM - 1; i <= SINGLE_TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.ASCENDING::lt,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.DESCENDING::lt,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.ASCENDING::lt,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::lt,
+          BoundaryOrder.DESCENDING::lt,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed lt on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+  @Test
+  public void testLtEq() {
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.ASCENDING::ltEq,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.DESCENDING::ltEq,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = SINGLE_FROM - 1; i <= SINGLE_TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.ASCENDING::ltEq,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.DESCENDING::ltEq,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.ASCENDING::ltEq,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::ltEq,
+          BoundaryOrder.DESCENDING::ltEq,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed ltEq on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+  @Test
+  public void testNotEq() {
+    for (int i = -16; i <= 16; ++i) {
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.ASCENDING::notEq,
+          ASCENDING.createValueComparator(i));
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.DESCENDING::notEq,
+          DESCENDING.createValueComparator(i));
+    }
+    for (int i = FROM - 1; i <= TO + 1; ++i) {
+      ColumnIndexBase<?>.ValueComparator singleComparator = SINGLE.createValueComparator(i);
+      validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.ASCENDING::notEq,
+          singleComparator);
+      validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.DESCENDING::notEq,
+          singleComparator);
+    }
+    ExecStats stats = new ExecStats();
+    for (int i = RAND_FROM - 1; i <= RAND_TO + 1; ++i) {
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with ASCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.ASCENDING::notEq,
+          RAND_ASCENDING.createValueComparator(i)));
+      stats.add(validateOperator("Mismatching page indexes for value " + i + " with DESCENDING order",
+          BoundaryOrder.UNORDERED::notEq,
+          BoundaryOrder.DESCENDING::notEq,
+          RAND_DESCENDING.createValueComparator(i)));
+    }
+    LOGGER.info("Executed notEq on random data (page count: {}, values searched: {}):\n{}", RAND_COUNT,
+        RAND_TO - RAND_FROM + 2, stats);
+  }
+
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestColumnIndexBuilder.java b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestColumnIndexBuilder.java
new file mode 100644
index 0000000..5a3947c
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestColumnIndexBuilder.java
@@ -0,0 +1,1546 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static java.util.Arrays.asList;
+import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.booleanColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.eq;
+import static org.apache.parquet.filter2.predicate.FilterApi.floatColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.gt;
+import static org.apache.parquet.filter2.predicate.FilterApi.gtEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.intColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.longColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.lt;
+import static org.apache.parquet.filter2.predicate.FilterApi.ltEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.userDefined;
+import static org.apache.parquet.filter2.predicate.LogicalInverter.invert;
+import static org.apache.parquet.schema.OriginalType.DECIMAL;
+import static org.apache.parquet.schema.OriginalType.UINT_8;
+import static org.apache.parquet.schema.OriginalType.UTF8;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BOOLEAN;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.FLOAT;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT64;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Comparator;
+import java.util.List;
+
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.column.statistics.Statistics;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators.BinaryColumn;
+import org.apache.parquet.filter2.predicate.Operators.BooleanColumn;
+import org.apache.parquet.filter2.predicate.Operators.DoubleColumn;
+import org.apache.parquet.filter2.predicate.Operators.FloatColumn;
+import org.apache.parquet.filter2.predicate.Operators.IntColumn;
+import org.apache.parquet.filter2.predicate.Operators.LongColumn;
+import org.apache.parquet.filter2.predicate.UserDefinedPredicate;
+import org.apache.parquet.io.api.Binary;
+import org.apache.parquet.schema.PrimitiveType;
+import org.apache.parquet.schema.Types;
+import org.junit.Test;
+
+/**
+ * Tests for {@link ColumnIndexBuilder}.
+ */
+public class TestColumnIndexBuilder {
+
+  public static class BinaryDecimalIsNullOrZeroUdp extends UserDefinedPredicate<Binary> {
+    private static final Binary ZERO = decimalBinary("0.0");
+
+    @Override
+    public boolean keep(Binary value) {
+      return value == null || value.equals(ZERO);
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Binary> statistics) {
+      Comparator<Binary> cmp = statistics.getComparator();
+      return cmp.compare(statistics.getMin(), ZERO) > 0 || cmp.compare(statistics.getMax(), ZERO) < 0;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Binary> statistics) {
+      Comparator<Binary> cmp = statistics.getComparator();
+      return cmp.compare(statistics.getMin(), ZERO) == 0 && cmp.compare(statistics.getMax(), ZERO) == 0;
+    }
+  }
+
+  public static class BinaryUtf8StartsWithB extends UserDefinedPredicate<Binary> {
+    private static final Binary B = stringBinary("B");
+    private static final Binary C = stringBinary("C");
+
+    @Override
+    public boolean keep(Binary value) {
+      return value != null && value.length() > 0 && value.getBytesUnsafe()[0] == 'B';
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Binary> statistics) {
+      Comparator<Binary> cmp = statistics.getComparator();
+      return cmp.compare(statistics.getMin(), C) >= 0 || cmp.compare(statistics.getMax(), B) < 0;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Binary> statistics) {
+      Comparator<Binary> cmp = statistics.getComparator();
+      return cmp.compare(statistics.getMin(), B) >= 0 && cmp.compare(statistics.getMax(), C) < 0;
+    }
+  }
+
+  public static class BooleanIsTrueOrNull extends UserDefinedPredicate<Boolean> {
+    @Override
+    public boolean keep(Boolean value) {
+      return value == null || value;
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Boolean> statistics) {
+      return statistics.getComparator().compare(statistics.getMax(), true) != 0;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Boolean> statistics) {
+      return statistics.getComparator().compare(statistics.getMin(), true) == 0;
+    }
+  }
+
+  public static class DoubleIsInteger extends UserDefinedPredicate<Double> {
+    @Override
+    public boolean keep(Double value) {
+      return value != null && Math.floor(value) == value;
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Double> statistics) {
+      double min = statistics.getMin();
+      double max = statistics.getMax();
+      Comparator<Double> cmp = statistics.getComparator();
+      return cmp.compare(Math.floor(min), Math.floor(max)) == 0 && cmp.compare(Math.floor(min), min) != 0
+          && cmp.compare(Math.floor(max), max) != 0;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Double> statistics) {
+      double min = statistics.getMin();
+      double max = statistics.getMax();
+      Comparator<Double> cmp = statistics.getComparator();
+      return cmp.compare(min, max) == 0 && cmp.compare(Math.floor(min), min) == 0;
+    }
+  }
+
+  public static class FloatIsInteger extends UserDefinedPredicate<Float> {
+    private static float floor(float value) {
+      return (float) Math.floor(value);
+    }
+
+    @Override
+    public boolean keep(Float value) {
+      return value != null && Math.floor(value) == value;
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Float> statistics) {
+      float min = statistics.getMin();
+      float max = statistics.getMax();
+      Comparator<Float> cmp = statistics.getComparator();
+      return cmp.compare(floor(min), floor(max)) == 0 && cmp.compare(floor(min), min) != 0
+          && cmp.compare(floor(max), max) != 0;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Float> statistics) {
+      float min = statistics.getMin();
+      float max = statistics.getMax();
+      Comparator<Float> cmp = statistics.getComparator();
+      return cmp.compare(min, max) == 0 && cmp.compare(floor(min), min) == 0;
+    }
+  }
+
+  public static class IntegerIsDivisableWith3 extends UserDefinedPredicate<Integer> {
+    @Override
+    public boolean keep(Integer value) {
+      return value != null && value % 3 == 0;
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Integer> statistics) {
+      int min = statistics.getMin();
+      int max = statistics.getMax();
+      return min % 3 != 0 && max % 3 != 0 && max - min < 3;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Integer> statistics) {
+      int min = statistics.getMin();
+      int max = statistics.getMax();
+      return min == max && min % 3 == 0;
+    }
+  }
+
+  public static class LongIsDivisableWith3 extends UserDefinedPredicate<Long> {
+    @Override
+    public boolean keep(Long value) {
+      return value != null && value % 3 == 0;
+    }
+
+    @Override
+    public boolean canDrop(org.apache.parquet.filter2.predicate.Statistics<Long> statistics) {
+      long min = statistics.getMin();
+      long max = statistics.getMax();
+      return min % 3 != 0 && max % 3 != 0 && max - min < 3;
+    }
+
+    @Override
+    public boolean inverseCanDrop(org.apache.parquet.filter2.predicate.Statistics<Long> statistics) {
+      long min = statistics.getMin();
+      long max = statistics.getMax();
+      return min == max && min % 3 == 0;
+    }
+  }
+
+  @Test
+  public void testBuildBinaryDecimal() {
+    PrimitiveType type = Types.required(BINARY).as(DECIMAL).precision(12).scale(2).named("test_binary_decimal");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(BinaryColumnIndexBuilder.class));
+    assertNull(builder.build());
+    BinaryColumn col = binaryColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("-0.17"), decimalBinary("1234567890.12")));
+    builder.add(sb.stats(type, decimalBinary("-234.23"), null, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, decimalBinary("-9999293.23"), decimalBinary("2348978.45")));
+    builder.add(sb.stats(type, null, null, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("87656273")));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 0, 3, 3, 0, 4, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        decimalBinary("1234567890.12"),
+        decimalBinary("-234.23"),
+        null,
+        decimalBinary("2348978.45"),
+        null,
+        null,
+        decimalBinary("87656273"));
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        decimalBinary("-0.17"),
+        decimalBinary("-234.23"),
+        null,
+        decimalBinary("-9999293.23"),
+        null,
+        null,
+        decimalBinary("87656273"));
+    assertCorrectFiltering(columnIndex, eq(col, decimalBinary("0.0")), 1, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, decimalBinary("87656273")), 0, 1, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 4, 7);
+    assertCorrectFiltering(columnIndex, gt(col, decimalBinary("2348978.45")), 1);
+    assertCorrectFiltering(columnIndex, gtEq(col, decimalBinary("2348978.45")), 1, 4);
+    assertCorrectFiltering(columnIndex, lt(col, decimalBinary("-234.23")), 4);
+    assertCorrectFiltering(columnIndex, ltEq(col, decimalBinary("-234.23")), 2, 4);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryDecimalIsNullOrZeroUdp.class), 0, 1, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryDecimalIsNullOrZeroUdp.class)), 1, 2, 4, 7);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null));
+    builder.add(sb.stats(type, decimalBinary("-9999293.23"), decimalBinary("-234.23")));
+    builder.add(sb.stats(type, decimalBinary("-0.17"), decimalBinary("87656273")));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("87656273")));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("1234567890.12"), null, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 4, 0, 0, 2, 0, 2, 3, 3);
+    assertCorrectNullPages(columnIndex, true, false, false, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        decimalBinary("-234.23"),
+        decimalBinary("87656273"),
+        null,
+        decimalBinary("87656273"),
+        null,
+        decimalBinary("1234567890.12"),
+        null);
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        decimalBinary("-9999293.23"),
+        decimalBinary("-0.17"),
+        null,
+        decimalBinary("87656273"),
+        null,
+        decimalBinary("1234567890.12"),
+        null);
+    assertCorrectFiltering(columnIndex, eq(col, decimalBinary("87656273")), 2, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 3, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, decimalBinary("87656273")), 0, 1, 2, 3, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 4, 6);
+    assertCorrectFiltering(columnIndex, gt(col, decimalBinary("87656273")), 6);
+    assertCorrectFiltering(columnIndex, gtEq(col, decimalBinary("87656273")), 2, 4, 6);
+    assertCorrectFiltering(columnIndex, lt(col, decimalBinary("-0.17")), 1);
+    assertCorrectFiltering(columnIndex, ltEq(col, decimalBinary("-0.17")), 1, 2);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryDecimalIsNullOrZeroUdp.class), 0, 2, 3, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryDecimalIsNullOrZeroUdp.class)), 1, 2, 4, 6);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("1234567890.12"), null, null, null));
+    builder.add(sb.stats(type, null, null, null, null));
+    builder.add(sb.stats(type, decimalBinary("1234567890.12"), decimalBinary("87656273")));
+    builder.add(sb.stats(type, decimalBinary("987656273"), decimalBinary("-0.17")));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, decimalBinary("-234.23"), decimalBinary("-9999293.23")));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 3, 2, 3, 4, 0, 0, 2, 0);
+    assertCorrectNullPages(columnIndex, true, true, false, true, false, false, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        null,
+        decimalBinary("1234567890.12"),
+        null,
+        decimalBinary("1234567890.12"),
+        decimalBinary("987656273"),
+        null,
+        decimalBinary("-234.23"));
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        null,
+        decimalBinary("1234567890.12"),
+        null,
+        decimalBinary("87656273"),
+        decimalBinary("-0.17"),
+        null,
+        decimalBinary("-9999293.23"));
+    assertCorrectFiltering(columnIndex, eq(col, decimalBinary("1234567890.12")), 2, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, decimalBinary("0.0")), 0, 1, 2, 3, 4, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 2, 4, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, decimalBinary("1234567890.12")));
+    assertCorrectFiltering(columnIndex, gtEq(col, decimalBinary("1234567890.12")), 2, 4);
+    assertCorrectFiltering(columnIndex, lt(col, decimalBinary("-0.17")), 7);
+    assertCorrectFiltering(columnIndex, ltEq(col, decimalBinary("-0.17")), 5, 7);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryDecimalIsNullOrZeroUdp.class), 0, 1, 2, 3, 5, 6);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryDecimalIsNullOrZeroUdp.class)), 2, 4, 5, 7);
+  }
+
+  @Test
+  public void testBuildBinaryUtf8() {
+    PrimitiveType type = Types.required(BINARY).as(UTF8).named("test_binary_utf8");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(BinaryColumnIndexBuilder.class));
+    assertNull(builder.build());
+    BinaryColumn col = binaryColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, stringBinary("Jeltz"), stringBinary("Slartibartfast"), null, null));
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, stringBinary("Beeblebrox"), stringBinary("Prefect")));
+    builder.add(sb.stats(type, stringBinary("Dent"), stringBinary("Trilian"), null));
+    builder.add(sb.stats(type, stringBinary("Beeblebrox")));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 5, 2, 0, 1, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, true, true, false, false, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        stringBinary("Slartibartfast"),
+        null,
+        null,
+        stringBinary("Prefect"),
+        stringBinary("Trilian"),
+        stringBinary("Beeblebrox"),
+        null);
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        stringBinary("Jeltz"),
+        null,
+        null,
+        stringBinary("Beeblebrox"),
+        stringBinary("Dent"),
+        stringBinary("Beeblebrox"),
+        null);
+    assertCorrectFiltering(columnIndex, eq(col, stringBinary("Marvin")), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 5, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, stringBinary("Beeblebrox")), 0, 1, 2, 3, 4, 5, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, gt(col, stringBinary("Prefect")), 1, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, stringBinary("Prefect")), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, stringBinary("Dent")), 4, 6);
+    assertCorrectFiltering(columnIndex, ltEq(col, stringBinary("Dent")), 4, 5, 6);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryUtf8StartsWithB.class), 4, 6);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryUtf8StartsWithB.class)), 0, 1, 2, 3, 4, 5, 7);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, stringBinary("Beeblebrox"), stringBinary("Dent"), null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, stringBinary("Dent"), stringBinary("Jeltz")));
+    builder.add(sb.stats(type, stringBinary("Dent"), stringBinary("Prefect"), null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, stringBinary("Slartibartfast")));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 5, 0, 1, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, false, true, true, false, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        stringBinary("Dent"),
+        null,
+        null,
+        stringBinary("Jeltz"),
+        stringBinary("Prefect"),
+        null,
+        stringBinary("Slartibartfast"),
+        null);
+    assertCorrectValues(columnIndex.getMinValues(),
+        stringBinary("Beeblebrox"),
+        null,
+        null,
+        stringBinary("Dent"),
+        stringBinary("Dent"),
+        null,
+        stringBinary("Slartibartfast"),
+        null);
+    assertCorrectFiltering(columnIndex, eq(col, stringBinary("Jeltz")), 3, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 4, 5, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, stringBinary("Slartibartfast")), 0, 1, 2, 3, 4, 5, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 3, 4, 6);
+    assertCorrectFiltering(columnIndex, gt(col, stringBinary("Marvin")), 4, 6);
+    assertCorrectFiltering(columnIndex, gtEq(col, stringBinary("Marvin")), 4, 6);
+    assertCorrectFiltering(columnIndex, lt(col, stringBinary("Dent")), 0);
+    assertCorrectFiltering(columnIndex, ltEq(col, stringBinary("Dent")), 0, 3, 4);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryUtf8StartsWithB.class), 0);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryUtf8StartsWithB.class)), 0, 1, 2, 3, 4, 5, 6, 7);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, stringBinary("Slartibartfast")));
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, stringBinary("Prefect"), stringBinary("Jeltz"), null));
+    builder.add(sb.stats(type, stringBinary("Dent"), stringBinary("Dent")));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, stringBinary("Dent"), stringBinary("Beeblebrox"), null, null));
+    assertEquals(8, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 0, 5, 1, 0, 2, 2, 2);
+    assertCorrectNullPages(columnIndex, true, false, true, false, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        stringBinary("Slartibartfast"),
+        null,
+        stringBinary("Prefect"),
+        stringBinary("Dent"),
+        null,
+        null,
+        stringBinary("Dent"));
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        stringBinary("Slartibartfast"),
+        null,
+        stringBinary("Jeltz"),
+        stringBinary("Dent"),
+        null,
+        null,
+        stringBinary("Beeblebrox"));
+    assertCorrectFiltering(columnIndex, eq(col, stringBinary("Marvin")), 3);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, stringBinary("Dent")), 0, 1, 2, 3, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 4, 7);
+    assertCorrectFiltering(columnIndex, gt(col, stringBinary("Prefect")), 1);
+    assertCorrectFiltering(columnIndex, gtEq(col, stringBinary("Prefect")), 1, 3);
+    assertCorrectFiltering(columnIndex, lt(col, stringBinary("Marvin")), 3, 4, 7);
+    assertCorrectFiltering(columnIndex, ltEq(col, stringBinary("Marvin")), 3, 4, 7);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryUtf8StartsWithB.class), 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryUtf8StartsWithB.class)), 0, 1, 2, 3, 4, 5, 6, 7);
+  }
+
+  @Test
+  public void testStaticBuildBinary() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(BINARY).as(UTF8).named("test_binary_utf8"),
+        BoundaryOrder.ASCENDING,
+        asList(true, true, false, false, true, false, true, false),
+        asList(1l, 2l, 3l, 4l, 5l, 6l, 7l, 8l),
+        toBBList(
+            null,
+            null,
+            stringBinary("Beeblebrox"),
+            stringBinary("Dent"),
+            null,
+            stringBinary("Jeltz"),
+            null,
+            stringBinary("Slartibartfast")),
+        toBBList(
+            null,
+            null,
+            stringBinary("Dent"),
+            stringBinary("Dent"),
+            null,
+            stringBinary("Prefect"),
+            null,
+            stringBinary("Slartibartfast")));
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectNullPages(columnIndex, true, true, false, false, true, false, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        null,
+        stringBinary("Dent"),
+        stringBinary("Dent"),
+        null,
+        stringBinary("Prefect"),
+        null,
+        stringBinary("Slartibartfast"));
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        null,
+        stringBinary("Beeblebrox"),
+        stringBinary("Dent"),
+        null,
+        stringBinary("Jeltz"),
+        null,
+        stringBinary("Slartibartfast"));
+  }
+
+  @Test
+  public void testFilterWithoutNullCounts() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(BINARY).as(UTF8).named("test_binary_utf8"),
+        BoundaryOrder.ASCENDING,
+        asList(true, true, false, false, true, false, true, false),
+        null,
+        toBBList(
+            null,
+            null,
+            stringBinary("Beeblebrox"),
+            stringBinary("Dent"),
+            null,
+            stringBinary("Jeltz"),
+            null,
+            stringBinary("Slartibartfast")),
+        toBBList(
+            null,
+            null,
+            stringBinary("Dent"),
+            stringBinary("Dent"),
+            null,
+            stringBinary("Prefect"),
+            null,
+            stringBinary("Slartibartfast")));
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertNull(columnIndex.getNullCounts());
+    assertCorrectNullPages(columnIndex, true, true, false, false, true, false, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(),
+        null,
+        null,
+        stringBinary("Dent"),
+        stringBinary("Dent"),
+        null,
+        stringBinary("Prefect"),
+        null,
+        stringBinary("Slartibartfast"));
+    assertCorrectValues(columnIndex.getMinValues(),
+        null,
+        null,
+        stringBinary("Beeblebrox"),
+        stringBinary("Dent"),
+        null,
+        stringBinary("Jeltz"),
+        null,
+        stringBinary("Slartibartfast"));
+
+    BinaryColumn col = binaryColumn("test_col");
+    assertCorrectFiltering(columnIndex, eq(col, stringBinary("Dent")), 2, 3);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, stringBinary("Dent")), 0, 1, 2, 3, 4, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 2, 3, 5, 7);
+    assertCorrectFiltering(columnIndex, userDefined(col, BinaryDecimalIsNullOrZeroUdp.class), 0, 1, 2, 3, 4, 5, 6, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BinaryDecimalIsNullOrZeroUdp.class)), 2, 3, 5, 7);
+  }
+
+  @Test
+  public void testBuildBoolean() {
+    PrimitiveType type = Types.required(BOOLEAN).named("test_boolean");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(BooleanColumnIndexBuilder.class));
+    assertNull(builder.build());
+    BooleanColumn col = booleanColumn("test_col");
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, false, true));
+    builder.add(sb.stats(type, true, false, null));
+    builder.add(sb.stats(type, true, true, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, false, false));
+    assertEquals(5, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 0);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), true, true, true, null, false);
+    assertCorrectValues(columnIndex.getMinValues(), false, false, true, null, false);
+    assertCorrectFiltering(columnIndex, eq(col, true), 0, 1, 2);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, true), 0, 1, 2, 3, 4);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4);
+    assertCorrectFiltering(columnIndex, userDefined(col, BooleanIsTrueOrNull.class), 0, 1, 2, 3);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BooleanIsTrueOrNull.class)), 0, 1, 4);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, false, false));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, null, null, null, null));
+    builder.add(sb.stats(type, false, true, null));
+    builder.add(sb.stats(type, false, true, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    assertEquals(7, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 0, 3, 4, 1, 2, 3);
+    assertCorrectNullPages(columnIndex, true, false, true, true, false, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, false, null, null, true, true, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, false, null, null, false, false, null);
+    assertCorrectFiltering(columnIndex, eq(col, true), 4, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, true), 0, 1, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, BooleanIsTrueOrNull.class), 0, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BooleanIsTrueOrNull.class)), 1, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, true, true));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, null, null, null, null));
+    builder.add(sb.stats(type, true, false, null));
+    builder.add(sb.stats(type, false, false, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    assertEquals(7, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 0, 3, 4, 1, 2, 3);
+    assertCorrectNullPages(columnIndex, true, false, true, true, false, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, true, null, null, true, false, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, true, null, null, false, false, null);
+    assertCorrectFiltering(columnIndex, eq(col, true), 1, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, true), 0, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, BooleanIsTrueOrNull.class), 0, 1, 2, 3, 4, 5, 6);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, BooleanIsTrueOrNull.class)), 4, 5);
+  }
+
+  @Test
+  public void testStaticBuildBoolean() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(BOOLEAN).named("test_boolean"),
+        BoundaryOrder.DESCENDING,
+        asList(false, true, false, true, false, true),
+        asList(9l, 8l, 7l, 6l, 5l, 0l),
+        toBBList(false, null, false, null, true, null),
+        toBBList(true, null, false, null, true, null));
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 9, 8, 7, 6, 5, 0);
+    assertCorrectNullPages(columnIndex, false, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), true, null, false, null, true, null);
+    assertCorrectValues(columnIndex.getMinValues(), false, null, false, null, true, null);
+  }
+
+  @Test
+  public void testBuildDouble() {
+    PrimitiveType type = Types.required(DOUBLE).named("test_double");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(DoubleColumnIndexBuilder.class));
+    assertNull(builder.build());
+    DoubleColumn col = doubleColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -4.2, -4.1));
+    builder.add(sb.stats(type, -11.7, 7.0, null));
+    builder.add(sb.stats(type, 2.2, 2.2, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 1.9, 2.32));
+    builder.add(sb.stats(type, -21.0, 8.1));
+    assertEquals(6, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 0, 0);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), -4.1, 7.0, 2.2, null, 2.32, 8.1);
+    assertCorrectValues(columnIndex.getMinValues(), -4.2, -11.7, 2.2, null, 1.9, -21.0);
+    assertCorrectFiltering(columnIndex, eq(col, 0.0), 1, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, 2.2), 0, 1, 2, 3, 4, 5);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, gt(col, 2.2), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2.2), 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, -4.2), 1, 5);
+    assertCorrectFiltering(columnIndex, ltEq(col, -4.2), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, DoubleIsInteger.class), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, DoubleIsInteger.class)), 0, 1, 2, 3, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -532.3, -345.2, null, null));
+    builder.add(sb.stats(type, -234.7, -234.6, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, -234.6, 2.99999));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 3.0, 42.83));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 1, 2, 3, 0, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, false, true, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, -345.2, -234.6, null, null, 2.99999, null, 42.83, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, -532.3, -234.7, null, null, -234.6, null, 3.0, null);
+    assertCorrectFiltering(columnIndex, eq(col, 0.0), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 6, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, 0.0), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, 2.99999), 7);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2.99999), 5, 7);
+    assertCorrectFiltering(columnIndex, lt(col, -234.6), 1, 2);
+    assertCorrectFiltering(columnIndex, ltEq(col, -234.6), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, DoubleIsInteger.class), 1, 5, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, DoubleIsInteger.class)), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, 532.3, 345.2));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 234.7, 234.6, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 234.69, -2.99999));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -3.0, -42.83));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 5, 0, 3, 1, 2, 0, 2, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 532.3, null, 234.7, null, 234.69, null, null, -3.0);
+    assertCorrectValues(columnIndex.getMinValues(), null, 345.2, null, 234.6, null, -2.99999, null, null, -42.83);
+    assertCorrectFiltering(columnIndex, eq(col, 234.6), 3, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, 2.2), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, gt(col, 2.2), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 234.69), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, lt(col, -2.99999), 8);
+    assertCorrectFiltering(columnIndex, ltEq(col, -2.99999), 5, 8);
+    assertCorrectFiltering(columnIndex, userDefined(col, DoubleIsInteger.class), 1, 5, 8);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, DoubleIsInteger.class)), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+  }
+
+  @Test
+  public void testBuildDoubleZeroNaN() {
+    PrimitiveType type = Types.required(DOUBLE).named("test_double");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -1.0, -0.0));
+    builder.add(sb.stats(type, 0.0, 1.0));
+    builder.add(sb.stats(type, 1.0, 100.0));
+    ColumnIndex columnIndex = builder.build();
+    assertCorrectValues(columnIndex.getMinValues(), -1.0, -0.0, 1.0);
+    assertCorrectValues(columnIndex.getMaxValues(), 0.0, 1.0, 100.0);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    builder.add(sb.stats(type, -1.0, -0.0));
+    builder.add(sb.stats(type, 0.0, Double.NaN));
+    builder.add(sb.stats(type, 1.0, 100.0));
+    assertNull(builder.build());
+  }
+
+  @Test
+  public void testStaticBuildDouble() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(DOUBLE).named("test_double"),
+        BoundaryOrder.UNORDERED,
+        asList(false, false, false, false, false, false),
+        asList(0l, 1l, 2l, 3l, 4l, 5l),
+        toBBList(-1.0, -2.0, -3.0, -4.0, -5.0, -6.0),
+        toBBList(1.0, 2.0, 3.0, 4.0, 5.0, 6.0));
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 4, 5);
+    assertCorrectNullPages(columnIndex, false, false, false, false, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), 1.0, 2.0, 3.0, 4.0, 5.0, 6.0);
+    assertCorrectValues(columnIndex.getMinValues(), -1.0, -2.0, -3.0, -4.0, -5.0, -6.0);
+  }
+
+  @Test
+  public void testBuildFloat() {
+    PrimitiveType type = Types.required(FLOAT).named("test_float");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(FloatColumnIndexBuilder.class));
+    assertNull(builder.build());
+    FloatColumn col = floatColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -4.2f, -4.1f));
+    builder.add(sb.stats(type, -11.7f, 7.0f, null));
+    builder.add(sb.stats(type, 2.2f, 2.2f, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 1.9f, 2.32f));
+    builder.add(sb.stats(type, -21.0f, 8.1f));
+    assertEquals(6, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 0, 0);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), -4.1f, 7.0f, 2.2f, null, 2.32f, 8.1f);
+    assertCorrectValues(columnIndex.getMinValues(), -4.2f, -11.7f, 2.2f, null, 1.9f, -21.0f);
+    assertCorrectFiltering(columnIndex, eq(col, 0.0f), 1, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, 2.2f), 0, 1, 2, 3, 4, 5);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, gt(col, 2.2f), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2.2f), 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, 0.0f), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, ltEq(col, 1.9f), 0, 1, 4, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, FloatIsInteger.class), 1, 4, 5);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, FloatIsInteger.class)), 0, 1, 2, 3, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -532.3f, -345.2f, null, null));
+    builder.add(sb.stats(type, -300.6f, -234.7f, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, -234.6f, 2.99999f));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 3.0f, 42.83f));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 1, 2, 3, 0, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, false, true, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, -345.2f, -234.7f, null, null, 2.99999f, null, 42.83f, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, -532.3f, -300.6f, null, null, -234.6f, null, 3.0f, null);
+    assertCorrectFiltering(columnIndex, eq(col, 0.0f), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 6, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, 2.2f), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, 2.2f), 5, 7);
+    assertCorrectFiltering(columnIndex, gtEq(col, -234.7f), 2, 5, 7);
+    assertCorrectFiltering(columnIndex, lt(col, -234.6f), 1, 2);
+    assertCorrectFiltering(columnIndex, ltEq(col, -234.6f), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, FloatIsInteger.class), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, FloatIsInteger.class)), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, 532.3f, 345.2f));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 234.7f, 234.6f, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 234.6f, -2.99999f));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -3.0f, -42.83f));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 5, 0, 3, 1, 2, 0, 2, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 532.3f, null, 234.7f, null, 234.6f, null, null, -3.0f);
+    assertCorrectValues(columnIndex.getMinValues(), null, 345.2f, null, 234.6f, null, -2.99999f, null, null, -42.83f);
+    assertCorrectFiltering(columnIndex, eq(col, 234.65f), 3);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, 2.2f), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, gt(col, 2.2f), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2.2f), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, lt(col, 0.0f), 5, 8);
+    assertCorrectFiltering(columnIndex, ltEq(col, 0.0f), 5, 8);
+    assertCorrectFiltering(columnIndex, userDefined(col, FloatIsInteger.class), 1, 5, 8);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, FloatIsInteger.class)), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+  }
+
+  @Test
+  public void testBuildFloatZeroNaN() {
+    PrimitiveType type = Types.required(FLOAT).named("test_float");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -1.0f, -0.0f));
+    builder.add(sb.stats(type, 0.0f, 1.0f));
+    builder.add(sb.stats(type, 1.0f, 100.0f));
+    ColumnIndex columnIndex = builder.build();
+    assertCorrectValues(columnIndex.getMinValues(), -1.0f, -0.0f, 1.0f);
+    assertCorrectValues(columnIndex.getMaxValues(), 0.0f, 1.0f, 100.0f);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    builder.add(sb.stats(type, -1.0f, -0.0f));
+    builder.add(sb.stats(type, 0.0f, Float.NaN));
+    builder.add(sb.stats(type, 1.0f, 100.0f));
+    assertNull(builder.build());
+  }
+
+  @Test
+  public void testStaticBuildFloat() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(FLOAT).named("test_float"),
+        BoundaryOrder.ASCENDING,
+        asList(true, true, true, false, false, false),
+        asList(9l, 8l, 7l, 6l, 0l, 0l),
+        toBBList(null, null, null, -3.0f, -2.0f, 0.1f),
+        toBBList(null, null, null, -2.0f, 0.0f, 6.0f));
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 9, 8, 7, 6, 0, 0);
+    assertCorrectNullPages(columnIndex, true, true, true, false, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, null, null, -2.0f, 0.0f, 6.0f);
+    assertCorrectValues(columnIndex.getMinValues(), null, null, null, -3.0f, -2.0f, 0.1f);
+  }
+
+  @Test
+  public void testBuildInt32() {
+    PrimitiveType type = Types.required(INT32).named("test_int32");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(IntColumnIndexBuilder.class));
+    assertNull(builder.build());
+    IntColumn col = intColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -4, 10));
+    builder.add(sb.stats(type, -11, 7, null));
+    builder.add(sb.stats(type, 2, 2, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 1, 2));
+    builder.add(sb.stats(type, -21, 8));
+    assertEquals(6, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 0, 0);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), 10, 7, 2, null, 2, 8);
+    assertCorrectValues(columnIndex.getMinValues(), -4, -11, 2, null, 1, -21);
+    assertCorrectFiltering(columnIndex, eq(col, 2), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, 2), 0, 1, 2, 3, 4, 5);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, gt(col, 2), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, 2), 0, 1, 4, 5);
+    assertCorrectFiltering(columnIndex, ltEq(col, 2), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -532, -345, null, null));
+    builder.add(sb.stats(type, -500, -42, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, -42, 2));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 3, 42));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 1, 2, 3, 0, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, false, true, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, -345, -42, null, null, 2, null, 42, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, -532, -500, null, null, -42, null, 3, null);
+    assertCorrectFiltering(columnIndex, eq(col, 2), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 6, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, 2), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, 2), 7);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2), 5, 7);
+    assertCorrectFiltering(columnIndex, lt(col, 2), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, ltEq(col, 2), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5, 6, 7,
+        8);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, 532, 345));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 234, 42, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 42, -2));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -3, -42));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 5, 0, 3, 1, 2, 0, 2, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 532, null, 234, null, 42, null, null, -3);
+    assertCorrectValues(columnIndex.getMinValues(), null, 345, null, 42, null, -2, null, null, -42);
+    assertCorrectFiltering(columnIndex, eq(col, 2), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, 2), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, gt(col, 2), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, lt(col, 2), 5, 8);
+    assertCorrectFiltering(columnIndex, ltEq(col, 2), 5, 8);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5, 6, 7,
+        8);
+  }
+
+  @Test
+  public void testStaticBuildInt32() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(INT32).named("test_int32"),
+        BoundaryOrder.DESCENDING,
+        asList(false, false, false, true, true, true),
+        asList(0l, 10l, 0l, 3l, 5l, 7l),
+        toBBList(10, 8, 6, null, null, null),
+        toBBList(9, 7, 5, null, null, null));
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 10, 0, 3, 5, 7);
+    assertCorrectNullPages(columnIndex, false, false, false, true, true, true);
+    assertCorrectValues(columnIndex.getMaxValues(), 9, 7, 5, null, null, null);
+    assertCorrectValues(columnIndex.getMinValues(), 10, 8, 6, null, null, null);
+  }
+
+  @Test
+  public void testBuildUInt8() {
+    PrimitiveType type = Types.required(INT32).as(UINT_8).named("test_uint8");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(IntColumnIndexBuilder.class));
+    assertNull(builder.build());
+    IntColumn col = intColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, 4, 10));
+    builder.add(sb.stats(type, 11, 17, null));
+    builder.add(sb.stats(type, 2, 2, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 1, 0xFF));
+    builder.add(sb.stats(type, 0xEF, 0xFA));
+    assertEquals(6, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0, 1, 2, 3, 0, 0);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), 10, 17, 2, null, 0xFF, 0xFA);
+    assertCorrectValues(columnIndex.getMinValues(), 4, 11, 2, null, 1, 0xEF);
+    assertCorrectFiltering(columnIndex, eq(col, 2), 2, 4);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, 2), 0, 1, 2, 3, 4, 5);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, gt(col, 2), 0, 1, 4, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, 0xEF), 0, 1, 2, 4);
+    assertCorrectFiltering(columnIndex, ltEq(col, 0xEF), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 0, 1, 4, 5);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 0, 0, null, null));
+    builder.add(sb.stats(type, 0, 42, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 42, 0xEE));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 0xEF, 0xFF));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 1, 2, 3, 0, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, false, true, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 0, 42, null, null, 0xEE, null, 0xFF, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, 0, 0, null, null, 42, null, 0xEF, null);
+    assertCorrectFiltering(columnIndex, eq(col, 2), 2);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 6, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, 2), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, 0xEE), 7);
+    assertCorrectFiltering(columnIndex, gtEq(col, 0xEE), 5, 7);
+    assertCorrectFiltering(columnIndex, lt(col, 42), 1, 2);
+    assertCorrectFiltering(columnIndex, ltEq(col, 42), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5, 6, 7,
+        8);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, 0xFF, 0xFF));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 0xEF, 0xEA, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 0xEE, 42));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 41, 0));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 5, 0, 3, 1, 2, 0, 2, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 0xFF, null, 0xEF, null, 0xEE, null, null, 41);
+    assertCorrectValues(columnIndex.getMinValues(), null, 0xFF, null, 0xEA, null, 42, null, null, 0);
+    assertCorrectFiltering(columnIndex, eq(col, 0xAB), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, 0xFF), 0, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, gt(col, 0xFF));
+    assertCorrectFiltering(columnIndex, gtEq(col, 0xFF), 1);
+    assertCorrectFiltering(columnIndex, lt(col, 42), 8);
+    assertCorrectFiltering(columnIndex, ltEq(col, 42), 5, 8);
+    assertCorrectFiltering(columnIndex, userDefined(col, IntegerIsDivisableWith3.class), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, IntegerIsDivisableWith3.class)), 0, 2, 3, 4, 5, 6, 7,
+        8);
+  }
+
+  @Test
+  public void testBuildInt64() {
+    PrimitiveType type = Types.required(INT64).named("test_int64");
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    assertThat(builder, instanceOf(LongColumnIndexBuilder.class));
+    assertNull(builder.build());
+    LongColumn col = longColumn("test_col");
+
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(type, -4l, 10l));
+    builder.add(sb.stats(type, -11l, 7l, null));
+    builder.add(sb.stats(type, 2l, 2l, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 1l, 2l));
+    builder.add(sb.stats(type, -21l, 8l));
+    assertEquals(6, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    ColumnIndex columnIndex = builder.build();
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 0l, 1l, 2l, 3l, 0l, 0l);
+    assertCorrectNullPages(columnIndex, false, false, false, true, false, false);
+    assertCorrectValues(columnIndex.getMaxValues(), 10l, 7l, 2l, null, 2l, 8l);
+    assertCorrectValues(columnIndex.getMinValues(), -4l, -11l, 2l, null, 1l, -21l);
+    assertCorrectFiltering(columnIndex, eq(col, 0l), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 1, 2, 3);
+    assertCorrectFiltering(columnIndex, notEq(col, 0l), 0, 1, 2, 3, 4, 5);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, gt(col, 2l), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2l), 0, 1, 2, 4, 5);
+    assertCorrectFiltering(columnIndex, lt(col, -21l));
+    assertCorrectFiltering(columnIndex, ltEq(col, -21l), 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, LongIsDivisableWith3.class), 0, 1, 5);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, LongIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -532l, -345l, null, null));
+    builder.add(sb.stats(type, -234l, -42l, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, -42l, 2l));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -3l, 42l));
+    builder.add(sb.stats(type, null, null));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.ASCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 2, 2, 1, 2, 3, 0, 2, 0, 2);
+    assertCorrectNullPages(columnIndex, true, false, false, true, true, false, true, false, true);
+    assertCorrectValues(columnIndex.getMaxValues(), null, -345l, -42l, null, null, 2l, null, 42l, null);
+    assertCorrectValues(columnIndex.getMinValues(), null, -532l, -234l, null, null, -42l, null, -3l, null);
+    assertCorrectFiltering(columnIndex, eq(col, -42l), 2, 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 1, 2, 3, 4, 6, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, -42l), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, gt(col, 2l), 7);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2l), 5, 7);
+    assertCorrectFiltering(columnIndex, lt(col, -42l), 1, 2);
+    assertCorrectFiltering(columnIndex, ltEq(col, -42l), 1, 2, 5);
+    assertCorrectFiltering(columnIndex, userDefined(col, LongIsDivisableWith3.class), 1, 2, 5, 7);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, LongIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5, 6, 7,
+        8);
+
+    builder = ColumnIndexBuilder.getBuilder(type, Integer.MAX_VALUE);
+    sb = new StatsBuilder();
+    builder.add(sb.stats(type, null, null, null, null, null));
+    builder.add(sb.stats(type, 532l, 345l));
+    builder.add(sb.stats(type, null, null, null));
+    builder.add(sb.stats(type, 234l, 42l, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, 42l, -2l));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, null, null));
+    builder.add(sb.stats(type, -3l, -42l));
+    assertEquals(9, builder.getPageCount());
+    assertEquals(sb.getMinMaxSize(), builder.getMinMaxSize());
+    columnIndex = builder.build();
+    assertEquals(BoundaryOrder.DESCENDING, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 5, 0, 3, 1, 2, 0, 2, 2, 0);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false, true, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 532l, null, 234l, null, 42l, null, null, -3l);
+    assertCorrectValues(columnIndex.getMinValues(), null, 345l, null, 42l, null, -2l, null, null, -42l);
+    assertCorrectFiltering(columnIndex, eq(col, 0l), 5);
+    assertCorrectFiltering(columnIndex, eq(col, null), 0, 2, 3, 4, 6, 7);
+    assertCorrectFiltering(columnIndex, notEq(col, 0l), 0, 1, 2, 3, 4, 5, 6, 7, 8);
+    assertCorrectFiltering(columnIndex, notEq(col, null), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, gt(col, 2l), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, gtEq(col, 2l), 1, 3, 5);
+    assertCorrectFiltering(columnIndex, lt(col, -42l));
+    assertCorrectFiltering(columnIndex, ltEq(col, -42l), 8);
+    assertCorrectFiltering(columnIndex, userDefined(col, LongIsDivisableWith3.class), 1, 3, 5, 8);
+    assertCorrectFiltering(columnIndex, invert(userDefined(col, LongIsDivisableWith3.class)), 0, 1, 2, 3, 4, 5, 6, 7,
+        8);
+  }
+
+  @Test
+  public void testStaticBuildInt64() {
+    ColumnIndex columnIndex = ColumnIndexBuilder.build(
+        Types.required(INT64).named("test_int64"),
+        BoundaryOrder.UNORDERED,
+        asList(true, false, true, false, true, false),
+        asList(1l, 2l, 3l, 4l, 5l, 6l),
+        toBBList(null, 2l, null, 4l, null, 9l),
+        toBBList(null, 3l, null, 15l, null, 10l));
+    assertEquals(BoundaryOrder.UNORDERED, columnIndex.getBoundaryOrder());
+    assertCorrectNullCounts(columnIndex, 1, 2, 3, 4, 5, 6);
+    assertCorrectNullPages(columnIndex, true, false, true, false, true, false);
+    assertCorrectValues(columnIndex.getMaxValues(), null, 3l, null, 15l, null, 10l);
+    assertCorrectValues(columnIndex.getMinValues(), null, 2l, null, 4l, null, 9l);
+  }
+
+  @Test
+  public void testNoOpBuilder() {
+    ColumnIndexBuilder builder = ColumnIndexBuilder.getNoOpBuilder();
+    StatsBuilder sb = new StatsBuilder();
+    builder.add(sb.stats(Types.required(BINARY).as(UTF8).named("test_binary_utf8"), stringBinary("Jeltz"),
+        stringBinary("Slartibartfast"), null, null));
+    builder.add(sb.stats(Types.required(BOOLEAN).named("test_boolean"), true, true, null, null));
+    builder.add(sb.stats(Types.required(DOUBLE).named("test_double"), null, null, null));
+    builder.add(sb.stats(Types.required(INT32).named("test_int32"), null, null));
+    builder.add(sb.stats(Types.required(INT64).named("test_int64"), -234l, -42l, null));
+    assertEquals(0, builder.getPageCount());
+    assertEquals(0, builder.getMinMaxSize());
+    assertNull(builder.build());
+  }
+
+  private static List<ByteBuffer> toBBList(Binary... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Binary value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(value.toByteBuffer());
+      }
+    }
+    return buffers;
+  }
+
+  private static List<ByteBuffer> toBBList(Boolean... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Boolean value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(ByteBuffer.wrap(BytesUtils.booleanToBytes(value)));
+      }
+    }
+    return buffers;
+  }
+
+  private static List<ByteBuffer> toBBList(Double... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Double value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(ByteBuffer.wrap(BytesUtils.longToBytes(Double.doubleToLongBits(value))));
+      }
+    }
+    return buffers;
+  }
+
+  private static List<ByteBuffer> toBBList(Float... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Float value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(ByteBuffer.wrap(BytesUtils.intToBytes(Float.floatToIntBits(value))));
+      }
+    }
+    return buffers;
+  }
+
+  private static List<ByteBuffer> toBBList(Integer... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Integer value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(ByteBuffer.wrap(BytesUtils.intToBytes(value)));
+      }
+    }
+    return buffers;
+  }
+
+  private static List<ByteBuffer> toBBList(Long... values) {
+    List<ByteBuffer> buffers = new ArrayList<>(values.length);
+    for (Long value : values) {
+      if (value == null) {
+        buffers.add(ByteBuffer.allocate(0));
+      } else {
+        buffers.add(ByteBuffer.wrap(BytesUtils.longToBytes(value)));
+      }
+    }
+    return buffers;
+  }
+
+  private static Binary decimalBinary(String num) {
+    return Binary.fromConstantByteArray(new BigDecimal(num).unscaledValue().toByteArray());
+  }
+
+  private static Binary stringBinary(String str) {
+    return Binary.fromString(str);
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Binary... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Binary expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertArrayEquals("Invalid value for page " + i, expectedValue.getBytesUnsafe(), value.array());
+      }
+    }
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Boolean... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Boolean expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertEquals("The byte buffer should be 1 byte long for boolean", 1, value.remaining());
+        assertEquals("Invalid value for page " + i, expectedValue.booleanValue(), value.get(0) != 0);
+      }
+    }
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Double... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Double expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertEquals("The byte buffer should be 8 bytes long for double", 8, value.remaining());
+        assertTrue("Invalid value for page " + i, Double.compare(expectedValue.doubleValue(), value.getDouble(0)) == 0);
+      }
+    }
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Float... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Float expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertEquals("The byte buffer should be 4 bytes long for double", 4, value.remaining());
+        assertTrue("Invalid value for page " + i, Float.compare(expectedValue.floatValue(), value.getFloat(0)) == 0);
+      }
+    }
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Integer... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Integer expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertEquals("The byte buffer should be 4 bytes long for int32", 4, value.remaining());
+        assertEquals("Invalid value for page " + i, expectedValue.intValue(), value.getInt(0));
+      }
+    }
+  }
+
+  private static void assertCorrectValues(List<ByteBuffer> values, Long... expectedValues) {
+    assertEquals(expectedValues.length, values.size());
+    for (int i = 0; i < expectedValues.length; ++i) {
+      Long expectedValue = expectedValues[i];
+      ByteBuffer value = values.get(i);
+      if (expectedValue == null) {
+        assertFalse("The byte buffer should be empty for null pages", value.hasRemaining());
+      } else {
+        assertEquals("The byte buffer should be 8 bytes long for int64", 8, value.remaining());
+        assertEquals("Invalid value for page " + i, expectedValue.intValue(), value.getLong(0));
+      }
+    }
+  }
+
+  private static void assertCorrectNullCounts(ColumnIndex columnIndex, long... expectedNullCounts) {
+    List<Long> nullCounts = columnIndex.getNullCounts();
+    assertEquals(expectedNullCounts.length, nullCounts.size());
+    for (int i = 0; i < expectedNullCounts.length; ++i) {
+      assertEquals("Invalid null count at page " + i, expectedNullCounts[i], nullCounts.get(i).longValue());
+    }
+  }
+
+  private static void assertCorrectNullPages(ColumnIndex columnIndex, boolean... expectedNullPages) {
+    List<Boolean> nullPages = columnIndex.getNullPages();
+    assertEquals(expectedNullPages.length, nullPages.size());
+    for (int i = 0; i < expectedNullPages.length; ++i) {
+      assertEquals("Invalid null pages at page " + i, expectedNullPages[i], nullPages.get(i).booleanValue());
+    }
+  }
+
+  private static class StatsBuilder {
+    private long minMaxSize;
+
+    Statistics<?> stats(PrimitiveType type, Object... values) {
+      Statistics<?> stats = Statistics.createStats(type);
+      for (Object value : values) {
+        if (value == null) {
+          stats.incrementNumNulls();
+          continue;
+        }
+        switch (type.getPrimitiveTypeName()) {
+          case BINARY:
+          case FIXED_LEN_BYTE_ARRAY:
+          case INT96:
+            stats.updateStats((Binary) value);
+            break;
+          case BOOLEAN:
+            stats.updateStats((boolean) value);
+            break;
+          case DOUBLE:
+            stats.updateStats((double) value);
+            break;
+          case FLOAT:
+            stats.updateStats((float) value);
+            break;
+          case INT32:
+            stats.updateStats((int) value);
+            break;
+          case INT64:
+            stats.updateStats((long) value);
+            break;
+          default:
+            fail("Unsupported value type for stats: " + value.getClass());
+        }
+      }
+      if (stats.hasNonNullValue()) {
+        minMaxSize += stats.getMinBytes().length;
+        minMaxSize += stats.getMaxBytes().length;
+      }
+      return stats;
+    }
+
+    long getMinMaxSize() {
+      return minMaxSize;
+    }
+  }
+
+  private static void assertCorrectFiltering(ColumnIndex ci, FilterPredicate predicate, int... expectedIndexes) {
+    TestIndexIterator.assertEquals(predicate.accept(ci), expectedIndexes);
+  }
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestIndexIterator.java b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestIndexIterator.java
new file mode 100644
index 0000000..d9047f2
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestIndexIterator.java
@@ -0,0 +1,63 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static org.junit.Assert.assertArrayEquals;
+
+import java.util.Arrays;
+import java.util.PrimitiveIterator;
+
+import org.junit.Test;
+
+import it.unimi.dsi.fastutil.ints.IntArrayList;
+import it.unimi.dsi.fastutil.ints.IntList;
+
+/**
+ * Unit test for {@link IndexIterator}.
+ */
+public class TestIndexIterator {
+  @Test
+  public void testAll() {
+    assertEquals(IndexIterator.all(10), 0, 1, 2, 3, 4, 5, 6, 7, 8, 9);
+  }
+
+  @Test
+  public void testFilter() {
+    assertEquals(IndexIterator.filter(30, value -> value % 3 == 0), 0, 3, 6, 9, 12, 15, 18, 21, 24, 27);
+  }
+
+  @Test
+  public void testFilterTranslate() {
+    assertEquals(IndexIterator.filterTranslate(20, value -> value < 5, Math::negateExact), 0, -1, -2, -3, -4);
+  }
+
+  @Test
+  public void testRangeTranslate() {
+    assertEquals(IndexIterator.rangeTranslate(11, 18, i -> i - 10), 1, 2, 3, 4, 5, 6, 7, 8);
+  }
+
+  static void assertEquals(PrimitiveIterator.OfInt actualIt, int... expectedValues) {
+    IntList actualList = new IntArrayList();
+    actualIt.forEachRemaining((int value) -> actualList.add(value));
+    int[] actualValues = actualList.toIntArray();
+    assertArrayEquals(
+        "ExpectedValues: " + Arrays.toString(expectedValues) + " ActualValues: " + Arrays.toString(actualValues),
+        expectedValues, actualValues);
+  }
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestOffsetIndexBuilder.java b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestOffsetIndexBuilder.java
new file mode 100644
index 0000000..1e1275c
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/column/columnindex/TestOffsetIndexBuilder.java
@@ -0,0 +1,113 @@
+/*
+ * 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.parquet.internal.column.columnindex;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndexBuilder;
+import org.junit.Test;
+
+/**
+ * Tests for {@link OffsetIndexBuilder}.
+ */
+public class TestOffsetIndexBuilder {
+  @Test
+  public void testBuilderWithSizeAndRowCount() {
+    OffsetIndexBuilder builder = OffsetIndexBuilder.getBuilder();
+    assertNull(builder.build());
+    assertNull(builder.build(1234));
+
+    builder.add(1000, 10);
+    builder.add(2000, 19);
+    builder.add(3000, 27);
+    builder.add(1200, 9);
+    assertCorrectValues(builder.build(),
+        0, 1000, 0,
+        1000, 2000, 10,
+        3000, 3000, 29,
+        6000, 1200, 56);
+    assertCorrectValues(builder.build(10000),
+        10000, 1000, 0,
+        11000, 2000, 10,
+        13000, 3000, 29,
+        16000, 1200, 56);
+  }
+
+  @Test
+  public void testNoOpBuilderWithSizeAndRowCount() {
+    OffsetIndexBuilder builder = OffsetIndexBuilder.getNoOpBuilder();
+    builder.add(1, 2);
+    builder.add(3, 4);
+    builder.add(5, 6);
+    builder.add(7, 8);
+    assertNull(builder.build());
+    assertNull(builder.build(1000));
+  }
+
+  @Test
+  public void testBuilderWithOffsetSizeIndex() {
+    OffsetIndexBuilder builder = OffsetIndexBuilder.getBuilder();
+    assertNull(builder.build());
+    assertNull(builder.build(1234));
+
+    builder.add(1000, 10000, 0);
+    builder.add(22000, 12000, 100);
+    builder.add(48000, 22000, 211);
+    builder.add(90000, 30000, 361);
+    assertCorrectValues(builder.build(),
+        1000, 10000, 0,
+        22000, 12000, 100,
+        48000, 22000, 211,
+        90000, 30000, 361);
+    assertCorrectValues(builder.build(100000),
+        101000, 10000, 0,
+        122000, 12000, 100,
+        148000, 22000, 211,
+        190000, 30000, 361);
+  }
+
+  @Test
+  public void testNoOpBuilderWithOffsetSizeIndex() {
+    OffsetIndexBuilder builder = OffsetIndexBuilder.getNoOpBuilder();
+    builder.add(1, 2, 3);
+    builder.add(4, 5, 6);
+    builder.add(7, 8, 9);
+    builder.add(10, 11, 12);
+    assertNull(builder.build());
+    assertNull(builder.build(1000));
+  }
+
+  private void assertCorrectValues(OffsetIndex offsetIndex, long... offset_size_rowIndex_triplets) {
+    assertEquals(offset_size_rowIndex_triplets.length % 3, 0);
+    int pageCount = offset_size_rowIndex_triplets.length / 3;
+    assertEquals("Invalid pageCount", pageCount, offsetIndex.getPageCount());
+    for (int i = 0; i < pageCount; ++i) {
+      assertEquals("Invalid offsetIndex at page " + i, offset_size_rowIndex_triplets[3 * i],
+          offsetIndex.getOffset(i));
+      assertEquals("Invalid compressedPageSize at page " + i, offset_size_rowIndex_triplets[3 * i + 1],
+          offsetIndex.getCompressedPageSize(i));
+      assertEquals("Invalid firstRowIndex at page " + i, offset_size_rowIndex_triplets[3 * i + 2],
+          offsetIndex.getFirstRowIndex(i));
+      long expectedLastPageIndex = (i < pageCount - 1) ? (offset_size_rowIndex_triplets[3 * i + 5] - 1) : 999;
+      assertEquals("Invalid lastRowIndex at page " + i, expectedLastPageIndex, offsetIndex.getLastRowIndex(i, 1000));
+    }
+  }
+}
diff --git a/parquet-column/src/test/java/org/apache/parquet/internal/filter2/columnindex/TestColumnIndexFilter.java b/parquet-column/src/test/java/org/apache/parquet/internal/filter2/columnindex/TestColumnIndexFilter.java
new file mode 100644
index 0000000..ae27214
--- /dev/null
+++ b/parquet-column/src/test/java/org/apache/parquet/internal/filter2/columnindex/TestColumnIndexFilter.java
@@ -0,0 +1,464 @@
+/*
+ * 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.parquet.internal.filter2.columnindex;
+
+import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.parquet.filter2.predicate.FilterApi.and;
+import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.booleanColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.eq;
+import static org.apache.parquet.filter2.predicate.FilterApi.gt;
+import static org.apache.parquet.filter2.predicate.FilterApi.gtEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.intColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.lt;
+import static org.apache.parquet.filter2.predicate.FilterApi.ltEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.or;
+import static org.apache.parquet.filter2.predicate.FilterApi.userDefined;
+import static org.apache.parquet.filter2.predicate.LogicalInverter.invert;
+import static org.apache.parquet.internal.column.columnindex.BoundaryOrder.ASCENDING;
+import static org.apache.parquet.internal.column.columnindex.BoundaryOrder.DESCENDING;
+import static org.apache.parquet.internal.column.columnindex.BoundaryOrder.UNORDERED;
+import static org.apache.parquet.internal.filter2.columnindex.ColumnIndexFilter.calculateRowRanges;
+import static org.apache.parquet.io.api.Binary.fromString;
+import static org.apache.parquet.schema.OriginalType.UTF8;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.BINARY;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.DOUBLE;
+import static org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.INT32;
+import static org.apache.parquet.schema.Types.optional;
+import static org.junit.Assert.assertArrayEquals;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.stream.LongStream;
+
+import org.apache.parquet.bytes.BytesUtils;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.predicate.Statistics;
+import org.apache.parquet.filter2.predicate.UserDefinedPredicate;
+import org.apache.parquet.hadoop.metadata.ColumnPath;
+import org.apache.parquet.internal.column.columnindex.BoundaryOrder;
+import org.apache.parquet.internal.column.columnindex.ColumnIndex;
+import org.apache.parquet.internal.column.columnindex.ColumnIndexBuilder;
+import org.apache.parquet.internal.column.columnindex.OffsetIndex;
+import org.apache.parquet.internal.column.columnindex.OffsetIndexBuilder;
+import org.apache.parquet.internal.column.columnindex.TestColumnIndexBuilder.BinaryUtf8StartsWithB;
+import org.apache.parquet.internal.column.columnindex.TestColumnIndexBuilder.DoubleIsInteger;
+import org.apache.parquet.internal.column.columnindex.TestColumnIndexBuilder.IntegerIsDivisableWith3;
+import org.apache.parquet.schema.PrimitiveType;
+import org.junit.Test;
+
+import it.unimi.dsi.fastutil.longs.LongArrayList;
+import it.unimi.dsi.fastutil.longs.LongList;
+
+/**
+ * Unit tests of {@link ColumnIndexFilter}
+ */
+public class TestColumnIndexFilter {
+  private static class CIBuilder {
+    private static final ByteBuffer EMPTY = ByteBuffer.wrap(new byte[0]);
+    private final PrimitiveType type;
+    private final BoundaryOrder order;
+    private List<Boolean> nullPages = new ArrayList<>();
+    private List<Long> nullCounts = new ArrayList<>();
+    private List<ByteBuffer> minValues = new ArrayList<>();
+    private List<ByteBuffer> maxValues = new ArrayList<>();
+
+    CIBuilder(PrimitiveType type, BoundaryOrder order) {
+      this.type = type;
+      this.order = order;
+    }
+
+    CIBuilder addNullPage(long nullCount) {
+      nullPages.add(true);
+      nullCounts.add(nullCount);
+      minValues.add(EMPTY);
+      maxValues.add(EMPTY);
+      return this;
+    }
+
+    CIBuilder addPage(long nullCount, int min, int max) {
+      nullPages.add(false);
+      nullCounts.add(nullCount);
+      minValues.add(ByteBuffer.wrap(BytesUtils.intToBytes(min)));
+      maxValues.add(ByteBuffer.wrap(BytesUtils.intToBytes(max)));
+      return this;
+    }
+
+    CIBuilder addPage(long nullCount, String min, String max) {
+      nullPages.add(false);
+      nullCounts.add(nullCount);
+      minValues.add(ByteBuffer.wrap(min.getBytes(UTF_8)));
+      maxValues.add(ByteBuffer.wrap(max.getBytes(UTF_8)));
+      return this;
+    }
+
+    CIBuilder addPage(long nullCount, double min, double max) {
+      nullPages.add(false);
+      nullCounts.add(nullCount);
+      minValues.add(ByteBuffer.wrap(BytesUtils.longToBytes(Double.doubleToLongBits(min))));
+      maxValues.add(ByteBuffer.wrap(BytesUtils.longToBytes(Double.doubleToLongBits(max))));
+      return this;
+    }
+
+    ColumnIndex build() {
+      return ColumnIndexBuilder.build(type, order, nullPages, nullCounts, minValues, maxValues);
+    }
+  }
+
+  private static class OIBuilder {
+    private final OffsetIndexBuilder builder = OffsetIndexBuilder.getBuilder();
+
+    OIBuilder addPage(long rowCount) {
+      builder.add(1234, rowCount);
+      return this;
+    }
+
+    OffsetIndex build() {
+      return builder.build();
+    }
+  }
+
+  public static class AnyInt extends UserDefinedPredicate<Integer> {
+
+    @Override
+    public boolean keep(Integer value) {
+      return true;
+    }
+
+    @Override
+    public boolean canDrop(Statistics<Integer> statistics) {
+      return false;
+    }
+
+    @Override
+    public boolean inverseCanDrop(Statistics<Integer> statistics) {
+      return true;
+    }
+
+  }
+
+  /**
+   * <pre>
+   * row     column1        column2        column3        column4 (no column index)
+   *      ------0------  ------0------  ------0------  ------0------
+   * 0.   1              Zulu           2.03
+   *      ------1------  ------1------  ------1------  ------1------
+   * 1.   2              Yankee         4.67
+   * 2.   3              Xray           3.42
+   * 3.   4              Whiskey        8.71
+   *                     ------2------                 ------2------
+   * 4.   5              Victor         0.56
+   * 5.   6              Uniform        4.30
+   *                                    ------2------  ------3------
+   * 6.   null           null           null
+   *      ------2------                                ------4------
+   * 7.   7              Tango          3.50
+   *                     ------3------
+   * 8.   7              null           3.14
+   *      ------3------
+   * 9.   7              null           null
+   *                                    ------3------
+   * 10.  null           null           9.99
+   *                     ------4------
+   * 11.  8              Sierra         8.78
+   *                                                   ------5------
+   * 12.  9              Romeo          9.56
+   * 13.  10             Quebec         2.71
+   *      ------4------
+   * 14.  11             Papa           5.71
+   * 15.  12             Oscar          4.09
+   *                     ------5------  ------4------  ------6------
+   * 16.  13             November       null
+   * 17.  14             Mike           null
+   * 18.  15             Lima           0.36
+   * 19.  16             Kilo           2.94
+   * 20.  17             Juliett        4.23
+   *      ------5------  ------6------                 ------7------
+   * 21.  18             India          null
+   * 22.  19             Hotel          5.32
+   *                                    ------5------
+   * 23.  20             Golf           4.17
+   * 24.  21             Foxtrot        7.92
+   * 25.  22             Echo           7.95
+   *                                   ------6------
+   * 26.  23             Delta          null
+   *      ------6------
+   * 27.  24             Charlie        null
+   *                                                   ------8------
+   * 28.  25             Bravo          null
+   *                     ------7------
+   * 29.  26             Alfa           null
+   * </pre>
+   */
+  private static final long TOTAL_ROW_COUNT = 30;
+  private static final ColumnIndex COLUMN1_CI = new CIBuilder(optional(INT32).named("column1"), ASCENDING)
+      .addPage(0, 1, 1)
+      .addPage(1, 2, 6)
+      .addPage(0, 7, 7)
+      .addPage(1, 7, 10)
+      .addPage(0, 11, 17)
+      .addPage(0, 18, 23)
+      .addPage(0, 24, 26)
+      .build();
+  private static final OffsetIndex COLUMN1_OI = new OIBuilder()
+      .addPage(1)
+      .addPage(6)
+      .addPage(2)
+      .addPage(5)
+      .addPage(7)
+      .addPage(6)
+      .addPage(3)
+      .build();
+  private static final ColumnIndex COLUMN2_CI = new CIBuilder(optional(BINARY).as(UTF8).named("column2"), DESCENDING)
+      .addPage(0, "Zulu", "Zulu")
+      .addPage(0, "Whiskey", "Yankee")
+      .addPage(1, "Tango", "Victor")
+      .addNullPage(3)
+      .addPage(0, "Oscar", "Sierra")
+      .addPage(0, "Juliett", "November")
+      .addPage(0, "Bravo", "India")
+      .addPage(0, "Alfa", "Alfa")
+      .build();
+  private static final OffsetIndex COLUMN2_OI = new OIBuilder()
+      .addPage(1)
+      .addPage(3)
+      .addPage(4)
+      .addPage(3)
+      .addPage(5)
+      .addPage(5)
+      .addPage(8)
+      .addPage(1)
+      .build();
+  private static final ColumnIndex COLUMN3_CI = new CIBuilder(optional(DOUBLE).named("column3"), UNORDERED)
+      .addPage(0, 2.03, 2.03)
+      .addPage(0, 0.56, 8.71)
+      .addPage(2, 3.14, 3.50)
+      .addPage(0, 2.71, 9.99)
+      .addPage(3, 0.36, 5.32)
+      .addPage(0, 4.17, 7.95)
+      .addNullPage(4)
+      .build();
+  private static final OffsetIndex COLUMN3_OI = new OIBuilder()
+      .addPage(1)
+      .addPage(5)
+      .addPage(4)
+      .addPage(6)
+      .addPage(7)
+      .addPage(3)
+      .addPage(4)
+      .build();
+  private static final ColumnIndex COLUMN4_CI = null;
+  private static final OffsetIndex COLUMN4_OI = new OIBuilder()
+      .addPage(1)
+      .addPage(3)
+      .addPage(2)
+      .addPage(1)
+      .addPage(5)
+      .addPage(4)
+      .addPage(5)
+      .addPage(7)
+      .addPage(2)
+      .build();
+  private static final ColumnIndexStore STORE = new ColumnIndexStore() {
+    @Override
+    public ColumnIndex getColumnIndex(ColumnPath column) {
+      switch (column.toDotString()) {
+        case "column1":
+          return COLUMN1_CI;
+        case "column2":
+          return COLUMN2_CI;
+        case "column3":
+          return COLUMN3_CI;
+        case "column4":
+          return COLUMN4_CI;
+        default:
+          return null;
+      }
+    }
+
+    @Override
+    public OffsetIndex getOffsetIndex(ColumnPath column) {
+      switch (column.toDotString()) {
+        case "column1":
+          return COLUMN1_OI;
+        case "column2":
+          return COLUMN2_OI;
+        case "column3":
+          return COLUMN3_OI;
+        case "column4":
+          return COLUMN4_OI;
+        default:
+          throw new MissingOffsetIndexException(column);
+      }
+    }
+  };
+
+  private static Set<ColumnPath> paths(String... columns) {
+    Set<ColumnPath> paths = new HashSet<>();
+    for (String column : columns) {
+      paths.add(ColumnPath.fromDotString(column));
+    }
+    return paths;
+  }
+
+  private static void assertAllRows(RowRanges ranges, long rowCount) {
+    LongList actualList = new LongArrayList();
+    ranges.iterator().forEachRemaining((long value) -> actualList.add(value));
+    LongList expectedList = new LongArrayList();
+    LongStream.range(0, rowCount).forEach(expectedList::add);
+    assertArrayEquals(expectedList + " != " + actualList, expectedList.toLongArray(), actualList.toLongArray());
+  }
+
+  private static void assertRows(RowRanges ranges, long... expectedRows) {
+    LongList actualList = new LongArrayList();
+    ranges.iterator().forEachRemaining((long value) -> actualList.add(value));
+    assertArrayEquals(Arrays.toString(expectedRows) + " != " + actualList, expectedRows, actualList.toLongArray());
+  }
+
+  @Test
+  public void testFiltering() {
+    Set<ColumnPath> paths = paths("column1", "column2", "column3", "column4");
+
+    assertAllRows(
+        calculateRowRanges(FilterCompat.get(
+            userDefined(intColumn("column1"), AnyInt.class)), STORE, paths, TOTAL_ROW_COUNT),
+        TOTAL_ROW_COUNT);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            and(
+                eq(intColumn("column1"), null),
+                eq(binaryColumn("column2"), null)),
+            and(
+                eq(doubleColumn("column3"), null),
+                eq(booleanColumn("column4"), null)))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        6, 9);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            and(
+                notEq(intColumn("column1"), null),
+                notEq(binaryColumn("column2"), null)),
+            and(
+                notEq(doubleColumn("column3"), null),
+                notEq(booleanColumn("column4"), null)))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        0, 1, 2, 3, 4, 5, 6, 7, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20, 21, 22, 23, 24, 25);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        or(
+            and(
+                lt(intColumn("column1"), 20),
+                gtEq(binaryColumn("column2"), fromString("Quebec"))),
+            and(
+                gt(doubleColumn("column3"), 5.32),
+                ltEq(binaryColumn("column4"), fromString("XYZ"))))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        0, 1, 2, 3, 4, 5, 6, 7, 10, 11, 12, 13, 14, 15, 23, 24, 25);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            and(
+                gtEq(intColumn("column1"), 7),
+                gt(binaryColumn("column2"), fromString("India"))),
+            and(
+                eq(doubleColumn("column3"), null),
+                notEq(binaryColumn("column4"), null)))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        7, 16, 17, 18, 19, 20);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            or(
+                invert(userDefined(intColumn("column1"), AnyInt.class)),
+                eq(binaryColumn("column2"), fromString("Echo"))),
+            eq(doubleColumn("column3"), 6.0))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        23, 24, 25);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            userDefined(intColumn("column1"), IntegerIsDivisableWith3.class),
+            and(
+                userDefined(binaryColumn("column2"), BinaryUtf8StartsWithB.class),
+                userDefined(doubleColumn("column3"), DoubleIsInteger.class)))),
+        STORE, paths, TOTAL_ROW_COUNT),
+        21, 22, 23, 24, 25);
+    assertRows(calculateRowRanges(FilterCompat.get(
+        and(
+            and(
... 3733 lines suppressed ...