You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2023/06/22 23:01:53 UTC

[calcite] 01/02: Refactor: Use PairList

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

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

commit 5e88fe0d1e24500707d928b7143282178ee8b35b
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Sat Jun 17 22:25:45 2023 -0700

    Refactor: Use PairList
    
    This change changes existing code to use PairList.
    
    (PairList was added in [CALCITE-5706]. Compared to an
    ArrayList whose elements are Pair objects, a PairList uses
    less memory (a single backing array, and no Pair objects),
    has a more concise API to add elements and convert to a Map.)
    
    Make Util.quotientList return a list that implements
    interface RandomAccess. (This should make some algorithms
    run faster.)
    
    Tune SqlValidatorUtil.uniquify by quickly checking whether
    the names are already unique and not null.
    
    Add class MapEntry, a simple implementation of
    java.util.Map.Entry. Unlike Pair it is immutable and has few
    dependencies. Unlike IdPair its hashCode is consistent with
    Map.Entry.
    
    Make Pair.equals to be consistent with Map.Entry, such as
    when the argument is a Map.Entry but not a Pair.
---
 .../calcite/adapter/enumerable/EnumUtils.java      |  23 +-
 .../java/org/apache/calcite/plan/RelOptUtil.java   | 110 ++++----
 .../apache/calcite/plan/RexImplicationChecker.java |  22 +-
 .../apache/calcite/plan/SubstitutionVisitor.java   |  14 +-
 .../org/apache/calcite/rel/AbstractRelNode.java    |  39 ++-
 .../AggregateExpandDistinctAggregatesRule.java     |  11 +-
 .../calcite/rel/type/RelDataTypeFactoryImpl.java   |  25 +-
 .../java/org/apache/calcite/rex/RexInputRef.java   |  12 +
 .../java/org/apache/calcite/runtime/MapEntry.java  |  84 ++++++
 .../java/org/apache/calcite/runtime/PairList.java  | 181 ++++++++++++-
 .../org/apache/calcite/sql/fun/SqlRowOperator.java |  24 +-
 .../calcite/sql/validate/SqlValidatorImpl.java     |  96 +++----
 .../calcite/sql/validate/SqlValidatorUtil.java     |  44 ++--
 .../validate/implicit/AbstractTypeCoercion.java    |   6 +-
 .../apache/calcite/sql2rel/RelDecorrelator.java    |  69 +++--
 .../sql2rel/RelStructuredTypeFlattener.java        |  83 +++---
 .../apache/calcite/sql2rel/SqlToRelConverter.java  | 136 +++++-----
 .../java/org/apache/calcite/tools/RelBuilder.java  |  32 +--
 .../org/apache/calcite/util/BuiltInMethod.java     |   3 +
 .../main/java/org/apache/calcite/util/Pair.java    |  16 +-
 .../main/java/org/apache/calcite/util/Util.java    |  91 +++++--
 .../java/org/apache/calcite/util/PairListTest.java | 284 +++++++++++++++++----
 .../java/org/apache/calcite/util/UtilTest.java     |  34 +++
 .../innodb/InnodbToEnumerableConverter.java        |  30 ++-
 .../java/org/apache/calcite/linq4j/Nullness.java   |  19 +-
 .../apache/calcite/adapter/mongodb/MongoRel.java   |   9 +-
 .../mongodb/MongoToEnumerableConverter.java        |   2 +-
 .../calcite/adapter/splunk/SplunkPushDownRule.java |  17 +-
 .../apache/calcite/sql/test/AbstractSqlTester.java |  11 +-
 .../org/apache/calcite/test/DiffRepository.java    |   6 +-
 .../test/catalog/CompoundNameColumnResolver.java   |  21 +-
 .../calcite/test/catalog/MockCatalogReader.java    |  17 +-
 .../main/java/org/apache/calcite/util/Smalls.java  |   8 +-
 33 files changed, 1037 insertions(+), 542 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
index c4f97ae9d3..48eefc6619 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumUtils.java
@@ -47,6 +47,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgramBuilder;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.runtime.SortedMultiMap;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
@@ -1042,16 +1043,16 @@ public class EnumUtils {
         Object watermark =
             requireNonNull(current[indexOfWatermarkedColumn],
                 "element[indexOfWatermarkedColumn]");
-        List<Pair<Long, Long>> windows =
+        PairList<Long, Long> windows =
             hopWindows(SqlFunctions.toLong(watermark), emitFrequency,
                 windowSize, offset);
-        for (Pair<Long, Long> window : windows) {
+        windows.forEach((left, right) -> {
           @Nullable Object[] curWithWindow = new Object[current.length + 2];
           System.arraycopy(current, 0, curWithWindow, 0, current.length);
-          curWithWindow[current.length] = window.left;
-          curWithWindow[current.length + 1] = window.right;
+          curWithWindow[current.length] = left;
+          curWithWindow[current.length + 1] = right;
           list.offer(curWithWindow);
-        }
+        });
         return takeOne();
       }
     }
@@ -1073,14 +1074,16 @@ public class EnumUtils {
     }
   }
 
-  private static List<Pair<Long, Long>> hopWindows(
-      long tsMillis, long periodMillis, long sizeMillis, long offsetMillis) {
-    ArrayList<Pair<Long, Long>> ret = new ArrayList<>(Math.toIntExact(sizeMillis / periodMillis));
-    long lastStart = tsMillis - ((tsMillis + periodMillis - offsetMillis) % periodMillis);
+  private static PairList<Long, Long> hopWindows(long tsMillis,
+      long periodMillis, long sizeMillis, long offsetMillis) {
+    PairList<Long, Long> ret =
+        PairList.withCapacity(Math.toIntExact(sizeMillis / periodMillis));
+    long lastStart =
+        tsMillis - ((tsMillis + periodMillis - offsetMillis) % periodMillis);
     for (long start = lastStart;
          start > tsMillis - sizeMillis;
          start -= periodMillis) {
-      ret.add(new Pair<>(start, start + sizeMillis));
+      ret.add(start, start + sizeMillis);
     }
     return ret;
   }
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index fc1d016dd7..b9aa94d981 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -80,6 +80,7 @@ import org.apache.calcite.rex.RexToSqlNodeConverterImpl;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.runtime.CalciteContextException;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.ModifiableView;
 import org.apache.calcite.sql.SqlExplainFormat;
 import org.apache.calcite.sql.SqlExplainLevel;
@@ -726,23 +727,19 @@ public abstract class RelOptUtil {
         : "rename: field count mismatch: in=" + inputType
         + ", out" + outputType;
 
-    final List<Pair<RexNode, String>> renames = new ArrayList<>();
-    for (Pair<RelDataTypeField, RelDataTypeField> pair
-        : Pair.zip(inputFields, outputFields)) {
-      final RelDataTypeField inputField = pair.left;
-      final RelDataTypeField outputField = pair.right;
+    final PairList<RexNode, String> renames = PairList.of();
+    final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
+    Pair.forEach(inputFields, outputFields, (inputField, outputField) -> {
       assert inputField.getType().equals(outputField.getType());
-      final RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
       renames.add(
-          Pair.of(
-              rexBuilder.makeInputRef(inputField.getType(),
-                  inputField.getIndex()),
-              outputField.getName()));
-    }
+          rexBuilder.makeInputRef(inputField.getType(),
+              inputField.getIndex()),
+          outputField.getName());
+    });
     final RelBuilder relBuilder =
         RelFactories.LOGICAL_BUILDER.create(rel.getCluster(), null);
     return relBuilder.push(rel)
-        .project(Pair.left(renames), Pair.right(renames), true)
+        .project(renames.leftList(), renames.rightList(), true)
         .build();
   }
 
@@ -1959,23 +1956,22 @@ public abstract class RelOptUtil {
     // If no projection was passed in, or the number of desired projection
     // columns is the same as the number of columns returned from the
     // join, then no need to create a projection
-    if ((newProjectOutputSize > 0)
-        && (newProjectOutputSize < joinOutputFields.size())) {
-      final List<Pair<RexNode, String>> newProjects = new ArrayList<>();
+    if (newProjectOutputSize > 0
+        && newProjectOutputSize < joinOutputFields.size()) {
+      final PairList<RexNode, String> newProjects = PairList.of();
       final RelBuilder relBuilder =
           RelFactories.LOGICAL_BUILDER.create(joinRel.getCluster(), null);
       final RexBuilder rexBuilder = relBuilder.getRexBuilder();
       for (int fieldIndex : outputProj) {
         final RelDataTypeField field = joinOutputFields.get(fieldIndex);
         newProjects.add(
-            Pair.of(
-                rexBuilder.makeInputRef(field.getType(), fieldIndex),
-                field.getName()));
+            rexBuilder.makeInputRef(field.getType(), fieldIndex),
+            field.getName());
       }
 
       // Create a project rel on the output of the join.
       return relBuilder.push(joinRel)
-          .project(Pair.left(newProjects), Pair.right(newProjects), true)
+          .project(newProjects.leftList(), newProjects.rightList(), true)
           .build();
     }
 
@@ -2143,16 +2139,14 @@ public abstract class RelOptUtil {
     case DELETE:
     case UPDATE:
       return typeFactory.createStructType(
-          ImmutableList.of(
-              Pair.of(AvaticaConnection.ROWCOUNT_COLUMN_NAME,
-                  typeFactory.createSqlType(SqlTypeName.BIGINT))));
+          PairList.of(AvaticaConnection.ROWCOUNT_COLUMN_NAME,
+              typeFactory.createSqlType(SqlTypeName.BIGINT)));
     case EXPLAIN:
       return typeFactory.createStructType(
-          ImmutableList.of(
-              Pair.of(AvaticaConnection.PLAN_COLUMN_NAME,
-                  typeFactory.createSqlType(
-                      SqlTypeName.VARCHAR,
-                      RelDataType.PRECISION_NOT_SPECIFIED))));
+          PairList.of(AvaticaConnection.PLAN_COLUMN_NAME,
+              typeFactory.createSqlType(
+                  SqlTypeName.VARCHAR,
+                  RelDataType.PRECISION_NOT_SPECIFIED)));
     default:
       throw Util.unexpected(kind);
     }
@@ -3746,27 +3740,21 @@ public abstract class RelOptUtil {
               extraLeftExprs, extraRightExprs, relBuilder.getRexBuilder());
     }
 
+    final PairList<RexNode, @Nullable String> pairs = PairList.of();
     relBuilder.push(originalJoin.getLeft());
     if (!extraLeftExprs.isEmpty()) {
       final List<RelDataTypeField> fields =
           relBuilder.peek().getRowType().getFieldList();
-      final List<Pair<RexNode, @Nullable String>> pairs =
-          new AbstractList<Pair<RexNode, @Nullable String>>() {
-            @Override public int size() {
-              return leftCount + extraLeftExprs.size();
-            }
-
-            @Override public Pair<RexNode, @Nullable String> get(int index) {
-              if (index < leftCount) {
-                RelDataTypeField field = fields.get(index);
-                return Pair.of(
-                    new RexInputRef(index, field.getType()), field.getName());
-              } else {
-                return Pair.of(extraLeftExprs.get(index - leftCount), null);
-              }
-            }
-          };
-      relBuilder.project(Pair.left(pairs), Pair.right(pairs));
+      for (int i = 0, n = leftCount + extraLeftExprs.size(); i < n; i++) {
+        if (i < leftCount) {
+          RelDataTypeField field = fields.get(i);
+          pairs.add(new RexInputRef(i, field.getType()), field.getName());
+        } else {
+          pairs.add(extraLeftExprs.get(i - leftCount), null);
+        }
+      }
+      relBuilder.project(pairs.leftList(), pairs.rightList());
+      pairs.clear();
     }
 
     relBuilder.push(originalJoin.getRight());
@@ -3774,28 +3762,18 @@ public abstract class RelOptUtil {
       final List<RelDataTypeField> fields =
           relBuilder.peek().getRowType().getFieldList();
       final int newLeftCount = leftCount + extraLeftExprs.size();
-      final List<Pair<RexNode, @Nullable String>> pairs =
-          new AbstractList<Pair<RexNode, @Nullable String>>() {
-            @Override public int size() {
-              return rightCount + extraRightExprs.size();
-            }
-
-            @Override public Pair<RexNode, @Nullable String> get(int index) {
-              if (index < rightCount) {
-                RelDataTypeField field = fields.get(index);
-                return Pair.of(
-                    new RexInputRef(index, field.getType()),
-                    field.getName());
-              } else {
-                return Pair.of(
-                    RexUtil.shift(
-                        extraRightExprs.get(index - rightCount),
-                        -newLeftCount),
-                    null);
-              }
-            }
-          };
-      relBuilder.project(Pair.left(pairs), Pair.right(pairs));
+      for (int i = 0, n = rightCount + extraRightExprs.size(); i < n; i++) {
+        if (i < rightCount) {
+          RelDataTypeField field = fields.get(i);
+          pairs.add(new RexInputRef(i, field.getType()), field.getName());
+        } else {
+          pairs.add(
+              RexUtil.shift(extraRightExprs.get(i - rightCount), -newLeftCount),
+              null);
+        }
+      }
+      relBuilder.project(pairs.leftList(), pairs.rightList());
+      pairs.clear();
     }
 
     final RelNode right = relBuilder.build();
diff --git a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
index 3dff20b5a1..f5230dd091 100644
--- a/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
+++ b/core/src/main/java/org/apache/calcite/plan/RexImplicationChecker.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.util.Pair;
@@ -39,7 +40,6 @@ import com.google.common.collect.Sets;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -220,10 +220,8 @@ public class RexImplicationChecker {
       ImmutableSet.Builder<Pair<RexInputRef, @Nullable RexNode>> usageBuilder =
           ImmutableSet.builder();
       if (entry.getValue().usageList.size() > 0) {
-        for (final Pair<SqlOperator, @Nullable RexNode> pair
-            : entry.getValue().usageList) {
-          usageBuilder.add(Pair.of(entry.getKey(), pair.getValue()));
-        }
+        entry.getValue().usageList.rightList().forEach(v ->
+            usageBuilder.add(Pair.of(entry.getKey(), v)));
         usagesBuilder.add(usageBuilder.build());
       }
     }
@@ -315,8 +313,10 @@ public class RexImplicationChecker {
 
     for (Map.Entry<RexInputRef, InputRefUsage<SqlOperator, @Nullable RexNode>> entry
         : secondUsageMap.entrySet()) {
-      final InputRefUsage<SqlOperator, @Nullable RexNode> secondUsage = entry.getValue();
-      final List<Pair<SqlOperator, @Nullable RexNode>> secondUsageList = secondUsage.usageList;
+      final InputRefUsage<SqlOperator, @Nullable RexNode> secondUsage =
+          entry.getValue();
+      final PairList<SqlOperator, @Nullable RexNode> secondUsageList =
+          secondUsage.usageList;
       final int secondLen = secondUsageList.size();
 
       if (secondUsage.usageCount != secondLen || secondLen > 2) {
@@ -332,7 +332,8 @@ public class RexImplicationChecker {
         return false;
       }
 
-      final List<Pair<SqlOperator, @Nullable RexNode>> firstUsageList = firstUsage.usageList;
+      final PairList<SqlOperator, @Nullable RexNode> firstUsageList =
+          firstUsage.usageList;
       final int firstLen = firstUsageList.size();
 
       final SqlKind fKind = firstUsageList.get(0).getKey().getKind();
@@ -504,8 +505,7 @@ public class RexImplicationChecker {
         @Nullable RexNode literal) {
       final InputRefUsage<SqlOperator, @Nullable RexNode> inputRefUse =
           getUsageMap(inputRef);
-      Pair<SqlOperator, @Nullable RexNode> use = Pair.of(op, literal);
-      inputRefUse.usageList.add(use);
+      inputRefUse.usageList.add(op, literal);
     }
 
     private InputRefUsage<SqlOperator, @Nullable RexNode> getUsageMap(RexInputRef rex) {
@@ -525,7 +525,7 @@ public class RexImplicationChecker {
    * @param <T1> left type
    * @param <T2> right type */
   private static class InputRefUsage<T1, T2> {
-    private final List<Pair<T1, T2>> usageList = new ArrayList<>();
+    private final PairList<T1, T2> usageList = PairList.of();
     private int usageCount = 0;
   }
 }
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index 53c03f810a..3cd45f4556 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -53,6 +53,7 @@ import org.apache.calcite.rex.RexSimplify;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.fun.SqlLibraryOperators;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -77,7 +78,7 @@ import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.ArrayList;
 import java.util.Collection;
-import java.util.Collections;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
@@ -1514,15 +1515,16 @@ public class SubstitutionVisitor {
       }
       // Add Project if the mapping breaks order of fields in GroupSet
       if (!Mappings.keepsOrdering(mapping)) {
-        final List<Integer> posList = new ArrayList<>();
         final int fieldCount = aggregate2.rowType.getFieldCount();
-        final List<Pair<Integer, Integer>> pairs = new ArrayList<>();
+        final PairList<Integer, Integer> pairs = PairList.of();
         final List<Integer> groupings = aggregate2.groupSet.toList();
         for (int i = 0; i < groupings.size(); i++) {
-          pairs.add(Pair.of(mapping.getTarget(groupings.get(i)), i));
+          pairs.add(mapping.getTarget(groupings.get(i)), i);
         }
-        Collections.sort(pairs);
-        pairs.forEach(pair -> posList.add(pair.right));
+        pairs.sort(
+            Comparator.<Map.Entry<Integer, Integer>>comparingInt(Map.Entry::getKey)
+                .thenComparingInt(Map.Entry::getValue));
+        final List<Integer> posList = new ArrayList<>(pairs.rightList());
         for (int i = posList.size(); i < fieldCount; i++) {
           posList.add(i);
         }
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 37d12c0528..66949853a2 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -33,6 +33,7 @@ import org.apache.calcite.rel.metadata.MetadataFactory;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexShuttle;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
@@ -49,6 +50,7 @@ import org.checkerframework.dataflow.qual.Pure;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
@@ -368,16 +370,16 @@ public abstract class AbstractRelNode implements RelNode {
     if (!result) {
       return false;
     }
-    List<Pair<String, @Nullable Object>> items1 = this.getDigestItems();
-    List<Pair<String, @Nullable Object>> items2 = that.getDigestItems();
+    PairList<String, @Nullable Object> items1 = this.getDigestItems();
+    PairList<String, @Nullable Object> items2 = that.getDigestItems();
     if (items1.size() != items2.size()) {
       return false;
     }
     for (int i = 0; result && i < items1.size(); i++) {
-      Pair<String, @Nullable Object> attr1 = items1.get(i);
-      Pair<String, @Nullable Object> attr2 = items2.get(i);
-      if (attr1.right instanceof RelNode) {
-        result = ((RelNode) attr1.right).deepEquals(attr2.right);
+      Map.Entry<String, @Nullable Object> attr1 = items1.get(i);
+      Map.Entry<String, @Nullable Object> attr2 = items2.get(i);
+      if (attr1.getValue() instanceof RelNode) {
+        result = ((RelNode) attr1.getValue()).deepEquals(attr2.getValue());
       } else {
         result = attr1.equals(attr2);
       }
@@ -393,9 +395,8 @@ public abstract class AbstractRelNode implements RelNode {
   @API(since = "1.25", status = API.Status.MAINTAINED)
   @Override public int deepHashCode() {
     int result = 31 + getTraitSet().hashCode();
-    List<Pair<String, @Nullable Object>> items = this.getDigestItems();
-    for (Pair<String, @Nullable Object> item : items) {
-      Object value = item.right;
+    PairList<String, @Nullable Object> items = this.getDigestItems();
+    for (@Nullable Object value : items.rightList()) {
       final int h;
       if (value == null) {
         h = 0;
@@ -409,7 +410,7 @@ public abstract class AbstractRelNode implements RelNode {
     return result;
   }
 
-  private List<Pair<String, @Nullable Object>> getDigestItems() {
+  private PairList<String, @Nullable Object> getDigestItems() {
     RelDigestWriter rdw = new RelDigestWriter();
     explainTerms(rdw);
     if (this instanceof Hintable) {
@@ -466,8 +467,7 @@ public abstract class AbstractRelNode implements RelNode {
    * prohibited.
    */
   private static final class RelDigestWriter implements RelWriter {
-
-    private final List<Pair<String, @Nullable Object>> attrs = new ArrayList<>();
+    private final PairList<String, @Nullable Object> attrs = PairList.of();
 
     @Nullable String digest = null;
 
@@ -496,22 +496,21 @@ public abstract class AbstractRelNode implements RelNode {
       sb.append('.');
       sb.append(node.getTraitSet());
       sb.append('(');
-      int j = 0;
-      for (Pair<String, @Nullable Object> attr : attrs) {
-        if (j++ > 0) {
+      attrs.forEachIndexed((j, left, right) -> {
+        if (j > 0) {
           sb.append(',');
         }
-        sb.append(attr.left);
+        sb.append(left);
         sb.append('=');
-        if (attr.right instanceof RelNode) {
-          RelNode input = (RelNode) attr.right;
+        if (right instanceof RelNode) {
+          RelNode input = (RelNode) right;
           sb.append(input.getRelTypeName());
           sb.append('#');
           sb.append(input.getId());
         } else {
-          sb.append(attr.right);
+          sb.append(right);
         }
-      }
+      });
       sb.append(')');
       digest = sb.toString();
       return this;
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
index 41d8313e59..10d42318a8 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AggregateExpandDistinctAggregatesRule.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -881,12 +882,12 @@ public final class AggregateExpandDistinctAggregatesRule
       Aggregate aggregate, List<Integer> argList, int filterArg,
       Map<Integer, Integer> sourceOf) {
     relBuilder.push(aggregate.getInput());
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    final PairList<RexNode, String> projects = PairList.of();
     final List<RelDataTypeField> childFields =
         relBuilder.peek().getRowType().getFieldList();
     for (int i : aggregate.getGroupSet()) {
       sourceOf.put(i, projects.size());
-      projects.add(RexInputRef.of2(i, childFields));
+      RexInputRef.add2(projects, i, childFields);
     }
     for (Integer arg : argList) {
       if (filterArg >= 0) {
@@ -908,16 +909,16 @@ public final class AggregateExpandDistinctAggregatesRule
                 argRef.left,
                 rexBuilder.makeNullLiteral(argRef.left.getType()));
         sourceOf.put(arg, projects.size());
-        projects.add(Pair.of(condition, "i$" + argRef.right));
+        projects.add(condition, "i$" + argRef.right);
         continue;
       }
       if (sourceOf.get(arg) != null) {
         continue;
       }
       sourceOf.put(arg, projects.size());
-      projects.add(RexInputRef.of2(arg, childFields));
+      RexInputRef.add2(projects, arg, childFields);
     }
-    relBuilder.project(Pair.left(projects), Pair.right(projects));
+    relBuilder.project(projects.leftList(), projects.rightList());
 
     // Get the distinct values of the GROUP BY fields and the arguments
     // to the agg functions.
diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
index 87a1e50452..c69f542002 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
@@ -27,6 +27,7 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeMappingRule;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.cache.CacheBuilder;
@@ -196,26 +197,10 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
   }
 
   private RelDataType createStructType(
-      final List<? extends Map.Entry<String, RelDataType>> fieldList, boolean nullable) {
-    return canonize(StructKind.FULLY_QUALIFIED,
-        new AbstractList<String>() {
-          @Override public String get(int index) {
-            return fieldList.get(index).getKey();
-          }
-
-          @Override public int size() {
-            return fieldList.size();
-          }
-        },
-        new AbstractList<RelDataType>() {
-          @Override public RelDataType get(int index) {
-            return fieldList.get(index).getValue();
-          }
-
-          @Override public int size() {
-            return fieldList.size();
-          }
-        }, nullable);
+      final List<? extends Map.Entry<String, RelDataType>> fieldList,
+      boolean nullable) {
+    return canonize(StructKind.FULLY_QUALIFIED, Pair.left(fieldList),
+        Pair.right(fieldList), nullable);
   }
 
   @Override public @Nullable RelDataType leastRestrictive(
diff --git a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
index 2c8e6530bb..606ac807cc 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexInputRef.java
@@ -18,6 +18,7 @@ package org.apache.calcite.rex;
 
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.Pair;
 
@@ -105,6 +106,17 @@ public class RexInputRef extends RexSlot {
         field.getName());
   }
 
+  /**
+   * Adds to a PairList a reference to a given field in a list of fields.
+   */
+  public static void add2(PairList<RexNode, String> list,
+      int index,
+      List<RelDataTypeField> fields) {
+    final RelDataTypeField field = fields.get(index);
+    list.add(new RexInputRef(index, field.getType()),
+        field.getName());
+  }
+
   @Override public SqlKind getKind() {
     return SqlKind.INPUT_REF;
   }
diff --git a/core/src/main/java/org/apache/calcite/runtime/MapEntry.java b/core/src/main/java/org/apache/calcite/runtime/MapEntry.java
new file mode 100644
index 0000000000..8b1a3a1354
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/runtime/MapEntry.java
@@ -0,0 +1,84 @@
+/*
+ * 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.calcite.runtime;
+
+import org.checkerframework.checker.nullness.qual.Nullable;
+
+import java.util.Map;
+import java.util.Objects;
+
+/**
+ * Simple implementation of {@link Map.Entry}.
+ *
+ * <p>It is immutable.
+ *
+ * <p>Key and value may be null if their types allow.
+ *
+ * @param <T> Key type
+ * @param <U> Value type
+ */
+public class MapEntry<T, U> implements Map.Entry<T, U> {
+  final T t;
+  final U u;
+
+  /** Creates a MapEntry. */
+  public MapEntry(T t, U u) {
+    this.t = t;
+    this.u = u;
+  }
+
+  @Override public String toString() {
+    return "<" + t + ", " + u + ">";
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>Compares equal to any {@link Map.Entry} with the equal key and value.
+   */
+  @SuppressWarnings("unchecked")
+  @Override public boolean equals(@Nullable Object o) {
+    return this == o
+        || o instanceof Map.Entry
+        && Objects.equals(this.t, ((Map.Entry<T, U>) o).getKey())
+        && Objects.equals(this.u, ((Map.Entry<T, U>) o).getValue());
+  }
+
+  /**
+   * {@inheritDoc}
+   *
+   * <p>Computes hash code consistent with
+   * {@link Map.Entry#hashCode()}.
+   */
+  @Override public int hashCode() {
+    int keyHash = t == null ? 0 : t.hashCode();
+    int valueHash = u == null ? 0 : u.hashCode();
+    return keyHash ^ valueHash;
+  }
+
+  @Override public T getKey() {
+    return t;
+  }
+
+  @Override public U getValue() {
+    return u;
+  }
+
+  @Override public U setValue(U value) {
+    throw new UnsupportedOperationException("setValue");
+  }
+}
diff --git a/core/src/main/java/org/apache/calcite/runtime/PairList.java b/core/src/main/java/org/apache/calcite/runtime/PairList.java
index c3cfaf14e2..eac9f00602 100644
--- a/core/src/main/java/org/apache/calcite/runtime/PairList.java
+++ b/core/src/main/java/org/apache/calcite/runtime/PairList.java
@@ -16,19 +16,25 @@
  */
 package org.apache.calcite.runtime;
 
-import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
+import org.apache.calcite.linq4j.function.Functions;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Lists;
 
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.RandomAccess;
 import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+import java.util.function.BiPredicate;
+
+import static com.google.common.base.Preconditions.checkArgument;
 
 import static java.util.Objects.requireNonNull;
 
@@ -49,6 +55,26 @@ public class PairList<T, U> extends AbstractList<Map.Entry<T, U>> {
     return new PairList<>(new ArrayList<>());
   }
 
+  /** Creates a singleton PairList. */
+  @SuppressWarnings("RedundantCast")
+  public static <T, U> PairList<T, U> of(T t, U u) {
+    final List<@Nullable Object> list = new ArrayList<>();
+    list.add((Object) t);
+    list.add((Object) u);
+    return new PairList<>(list);
+  }
+
+  /** Creates a PairList with one or more entries. */
+  public static <T, U> PairList<T, U> copyOf(T t, U u, Object... rest) {
+    checkArgument(rest.length % 2 == 0, "even number");
+    return new PairList<>(new ArrayList<>(Lists.asList(t, u, rest)));
+  }
+
+  /** Creates an empty PairList with a specified initial capacity. */
+  public static <T, U> PairList<T, U> withCapacity(int initialCapacity) {
+    return backedBy(new ArrayList<>(initialCapacity));
+  }
+
   /** Creates a PairList backed by a given list.
    *
    * <p>Changes to the backing list will be reflected in the PairList.
@@ -68,16 +94,25 @@ public class PairList<T, U> extends AbstractList<Map.Entry<T, U>> {
     return new PairList<>(list);
   }
 
+  /** Creates a Builder. */
+  public static <T, U> Builder<T, U> builder() {
+    return new Builder<>();
+  }
+
   @SuppressWarnings("unchecked")
   @Override public Map.Entry<T, U> get(int index) {
     int x = index * 2;
-    return Pair.of((T) list.get(x), (U) list.get(x + 1));
+    return new MapEntry<>((T) list.get(x), (U) list.get(x + 1));
   }
 
   @Override public int size() {
     return list.size() / 2;
   }
 
+  @Override public void clear() {
+    list.clear();
+  }
+
   @SuppressWarnings("RedundantCast")
   @Override public boolean add(Map.Entry<T, U> entry) {
     list.add((Object) entry.getKey());
@@ -99,26 +134,79 @@ public class PairList<T, U> extends AbstractList<Map.Entry<T, U>> {
     list.add((Object) u);
   }
 
+  /** Adds a pair to this list at a given position. */
+  @SuppressWarnings("RedundantCast")
+  public void add(int index, T t, U u) {
+    int x = index * 2;
+    list.add(x, (Object) t);
+    list.add(x + 1, (Object) u);
+  }
+
+  /** Adds to this list the contents of another PairList.
+   *
+   * <p>Equivalent to {@link #addAll(Collection)}, but more efficient. */
+  public boolean addAll(PairList<T, U> list2) {
+    return list.addAll(list2.list);
+  }
+
+  /** Adds to this list, at a given index, the contents of another PairList.
+   *
+   * <p>Equivalent to {@link #addAll(int, Collection)}, but more efficient. */
+  public boolean addAll(int index, PairList<T, U> list2) {
+    int x = index * 2;
+    return list.addAll(x, list2.list);
+  }
+
+  @Override public Map.Entry<T, U> set(int index, Map.Entry<T, U> entry) {
+    return set(index, entry.getKey(), entry.getValue());
+  }
+
+  @SuppressWarnings("unchecked")
+  public Map.Entry<T, U> set(int index, T t, U u) {
+    int x = index * 2;
+    T t0 = (T) list.set(x, t);
+    U u0 = (U) list.set(x + 1, u);
+    return new MapEntry<>(t0, u0);
+  }
+
   @SuppressWarnings("unchecked")
   @Override public Map.Entry<T, U> remove(int index) {
     final int x = index * 2;
     T t = (T) list.remove(x);
     U u = (U) list.remove(x);
-    return Pair.of(t, u);
+    return new MapEntry<>(t, u);
   }
 
   /** Returns an unmodifiable list view consisting of the left entry of each
    * pair. */
   @SuppressWarnings("unchecked")
   public List<T> leftList() {
-    return Util.quotientList((List<T>) list, 2, 0);
+    final int size = list.size() / 2;
+    return new RandomAccessList<T>() {
+      @Override public int size() {
+        return size;
+      }
+
+      @Override public T get(int index) {
+        return (T) list.get(index * 2);
+      }
+    };
   }
 
   /** Returns an unmodifiable list view consisting of the right entry of each
    * pair. */
   @SuppressWarnings("unchecked")
   public List<U> rightList() {
-    return Util.quotientList((List<U>) list, 2, 1);
+    final int size = list.size() / 2;
+    return new RandomAccessList<U>() {
+      @Override public int size() {
+        return size;
+      }
+
+      @Override public U get(int index) {
+        return (U) list.get(index * 2 + 1);
+      }
+    };
   }
 
   /** Calls a BiConsumer with each pair in this list. */
@@ -158,6 +246,59 @@ public class PairList<T, U> extends AbstractList<Map.Entry<T, U>> {
     return backedBy(immutableList);
   }
 
+  /** Applies a mapping function to each element of this list. */
+  @SuppressWarnings("unchecked")
+  public <R> List<R> transform(BiFunction<T, U, R> function) {
+    return Functions.generate(list.size() / 2, index -> {
+      final int x = index * 2;
+      final T t = (T) list.get(x);
+      final U u = (U) list.get(x + 1);
+      return function.apply(t, u);
+    });
+  }
+
+  /** Returns whether the predicate is true for at least one pair
+   * in this list. */
+  @SuppressWarnings("unchecked")
+  public boolean anyMatch(BiPredicate<T, U> predicate) {
+    for (int i = 0; i < list.size();) {
+      final T t = (T) list.get(i++);
+      final U u = (U) list.get(i++);
+      if (predicate.test(t, u)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /** Returns whether the predicate is true for all pairs
+   * in this list. */
+  @SuppressWarnings("unchecked")
+  public boolean allMatch(BiPredicate<T, U> predicate) {
+    for (int i = 0; i < list.size();) {
+      final T t = (T) list.get(i++);
+      final U u = (U) list.get(i++);
+      if (!predicate.test(t, u)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  /** Returns whether the predicate is true for no pairs
+   * in this list. */
+  @SuppressWarnings("unchecked")
+  public boolean noMatch(BiPredicate<T, U> predicate) {
+    for (int i = 0; i < list.size();) {
+      final T t = (T) list.get(i++);
+      final U u = (U) list.get(i++);
+      if (predicate.test(t, u)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
   /** Action to be taken each step of an indexed iteration over a PairList.
    *
    * @param <T> First type
@@ -175,4 +316,32 @@ public class PairList<T, U> extends AbstractList<Map.Entry<T, U>> {
      */
     void accept(int index, T t, U u);
   }
+
+  /** Builds a PairList.
+   *
+   * @param <T> First type
+   * @param <U> Second type */
+  public static class Builder<T, U> {
+    final List<@Nullable Object> list = new ArrayList<>();
+
+    /** Adds a pair to the list under construction. */
+    @SuppressWarnings("RedundantCast")
+    public Builder<T, U> add(T t, U u) {
+      list.add((Object) t);
+      list.add((Object) u);
+      return this;
+    }
+
+    /** Builds the PairList. */
+    public PairList<T, U> build() {
+      return new PairList<>(list);
+    }
+  }
+
+  /** Base class for a list that implements {@link java.util.RandomAccess}.
+   *
+   * @param <E> Element type */
+  private abstract static class RandomAccessList<E>
+      extends AbstractList<E> implements RandomAccess {
+  }
 }
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
index 3e28ca8476..b20d381765 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlRowOperator.java
@@ -26,10 +26,6 @@ import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.sql.SqlWriter;
 import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.util.Pair;
-
-import java.util.AbstractList;
-import java.util.Map;
 
 /**
  * SqlRowOperator represents the special ROW constructor.
@@ -57,21 +53,15 @@ public class SqlRowOperator extends SqlSpecialOperator {
     // {e1type,e2type}.  According to the standard, field names are
     // implementation-defined.
     final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
-    final RelDataType recordType =
-        typeFactory.createStructType(
-            new AbstractList<Map.Entry<String, RelDataType>>() {
-              @Override public Map.Entry<String, RelDataType> get(int index) {
-                return Pair.of(SqlUtil.deriveAliasFromOrdinal(index),
-                    opBinding.getOperandType(index));
-              }
-
-              @Override public int size() {
-                return opBinding.getOperandCount();
-              }
-            });
+    final RelDataTypeFactory.Builder builder = typeFactory.builder();
+    for (int index = 0; index < opBinding.getOperandCount(); index++) {
+      builder.add(SqlUtil.deriveAliasFromOrdinal(index),
+          opBinding.getOperandType(index));
+    }
+    final RelDataType recordType = builder.build();
 
     // The value of ROW(e1,e2) is considered null if and only all of its
-    // fields (i.e., e1, e2) are null. Otherwise ROW can not be null.
+    // fields (i.e., e1, e2) are null. Otherwise, ROW can not be null.
     final boolean nullable =
         recordType.getFieldList().stream()
             .allMatch(f -> f.getType().isNullable());
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 432699123a..db6d937a70 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rex.RexVisitor;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.runtime.CalciteException;
 import org.apache.calcite.runtime.Feature;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.schema.Table;
@@ -381,7 +382,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   @Override public SqlNodeList expandStar(SqlNodeList selectList,
       SqlSelect select, boolean includeSystemVars) {
     final List<SqlNode> list = new ArrayList<>();
-    final List<Map.Entry<String, RelDataType>> types = new ArrayList<>();
+    final PairList<String, RelDataType> types = PairList.of();
     for (SqlNode selectItem : selectList) {
       final RelDataType originalType = getValidatedNodeTypeIfKnown(selectItem);
       expandSelectItem(selectItem, select, first(originalType, unknownType),
@@ -446,7 +447,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    */
   private boolean expandSelectItem(final SqlNode selectItem, SqlSelect select,
       RelDataType targetType, List<SqlNode> selectItems, Set<String> aliases,
-      List<Map.Entry<String, RelDataType>> fields, boolean includeSystemVars) {
+      PairList<String, RelDataType> fields, boolean includeSystemVars) {
     final SelectScope scope = (SelectScope) getWhereScope(select);
     if (expandStar(selectItems, aliases, fields, includeSystemVars, scope,
         selectItem)) {
@@ -492,12 +493,12 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       type = requireNonNull(selectScope.nullifyType(stripAs(expanded), type));
     }
     setValidatedNodeType(expanded, type);
-    fields.add(Pair.of(alias, type));
+    fields.add(alias, type);
     return false;
   }
 
-  private static SqlNode expandExprFromJoin(SqlJoin join, SqlIdentifier identifier,
-      @Nullable SelectScope scope) {
+  private static SqlNode expandExprFromJoin(SqlJoin join,
+      SqlIdentifier identifier, SelectScope scope) {
     if (join.getConditionType() != JoinConditionType.USING) {
       return identifier;
     }
@@ -517,13 +518,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         }
 
         assert qualifiedNode.size() == 2;
-        final SqlNode finalNode =
-            SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO,
-                SqlStdOperatorTable.COALESCE.createCall(SqlParserPos.ZERO,
-                    qualifiedNode.get(0),
-                    qualifiedNode.get(1)),
-                new SqlIdentifier(name, SqlParserPos.ZERO));
-        return finalNode;
+        return SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO,
+            SqlStdOperatorTable.COALESCE.createCall(SqlParserPos.ZERO,
+                qualifiedNode.get(0),
+                qualifiedNode.get(1)),
+            new SqlIdentifier(name, SqlParserPos.ZERO));
       }
     }
 
@@ -614,7 +613,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   private boolean expandStar(List<SqlNode> selectItems, Set<String> aliases,
-      List<Map.Entry<String, RelDataType>> fields, boolean includeSystemVars,
+      PairList<String, RelDataType> fields, boolean includeSystemVars,
       SelectScope scope, SqlNode node) {
     if (!(node instanceof SqlIdentifier)) {
       return false;
@@ -682,8 +681,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
             final RelDataType type = entry.getValue();
             if (!type.isNullable()) {
               fields.set(i,
-                  Pair.of(entry.getKey(),
-                      typeFactory.createTypeWithNullability(type, true)));
+                  entry.getKey(),
+                  typeFactory.createTypeWithNullability(type, true));
             }
           }
         }
@@ -763,7 +762,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   private boolean addOrExpandField(List<SqlNode> selectItems, Set<String> aliases,
-      List<Map.Entry<String, RelDataType>> fields, boolean includeSystemVars,
+      PairList<String, RelDataType> fields, boolean includeSystemVars,
       SelectScope scope, SqlIdentifier id, RelDataTypeField field) {
     switch (field.getType().getStructKind()) {
     case PEEK_FIELDS:
@@ -1226,10 +1225,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   private @Nullable SqlValidatorNamespace getNamespace(SqlNode node,
-      @Nullable SqlValidatorScope scope) {
+      SqlValidatorScope scope) {
     if (node instanceof SqlIdentifier && scope instanceof DelegatingScope) {
       final SqlIdentifier id = (SqlIdentifier) node;
-      final DelegatingScope idScope = (DelegatingScope) ((DelegatingScope) scope).getParent();
+      final DelegatingScope idScope =
+          (DelegatingScope) ((DelegatingScope) scope).getParent();
       return getNamespace(id, idScope);
     } else if (node instanceof SqlCall) {
       // Handle extended identifiers.
@@ -1322,7 +1322,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    */
   @API(since = "1.27", status = API.Status.INTERNAL)
   SqlValidatorNamespace getNamespaceOrThrow(SqlNode node,
-      @Nullable SqlValidatorScope scope) {
+      SqlValidatorScope scope) {
     return requireNonNull(
         getNamespace(node, scope),
         () -> "namespace for " + node + ", scope " + scope);
@@ -2132,7 +2132,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     if (!Objects.equals(alias, uniqueAlias)) {
       exp = SqlValidatorUtil.addAlias(exp, uniqueAlias);
     }
-    fieldList.add(Pair.of(uniqueAlias, deriveType(scope, exp)));
+    ((PairList<String, RelDataType>) fieldList)
+        .add(uniqueAlias, deriveType(scope, exp));
     list.add(exp);
   }
 
@@ -4605,7 +4606,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     final SqlValidatorScope selectScope = getSelectScope(select);
     final List<SqlNode> expandedSelectItems = new ArrayList<>();
     final Set<String> aliases = new HashSet<>();
-    final List<Map.Entry<String, RelDataType>> fieldList = new ArrayList<>();
+    final PairList<String, RelDataType> fieldList = PairList.of();
 
     for (SqlNode selectItem : selectItems) {
       if (selectItem instanceof SqlSelect) {
@@ -4696,7 +4697,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
    */
   private void handleScalarSubQuery(SqlSelect parentSelect,
       SqlSelect selectItem, List<SqlNode> expandedSelectItems,
-      Set<String> aliasList, List<Map.Entry<String, RelDataType>> fieldList) {
+      Set<String> aliasList, PairList<String, RelDataType> fieldList) {
     // A scalar sub-query only has one output column.
     if (1 != SqlNonNullableAccessors.getSelectList(selectItem).size()) {
       throw newValidationError(selectItem,
@@ -4723,7 +4724,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
 
     RelDataType nodeType = rec.getFieldList().get(0).getType();
     nodeType = typeFactory.createTypeWithNullability(nodeType, true);
-    fieldList.add(Pair.of(alias, nodeType));
+    fieldList.add(alias, nodeType);
   }
 
   /**
@@ -4744,12 +4745,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return baseRowType;
     }
     List<RelDataTypeField> targetFields = baseRowType.getFieldList();
-    final List<Map.Entry<String, RelDataType>> fields = new ArrayList<>();
+    final PairList<String, RelDataType> fields = PairList.of();
     if (append) {
       for (RelDataTypeField targetField : targetFields) {
-        fields.add(
-            Pair.of(SqlUtil.deriveAliasFromOrdinal(fields.size()),
-                targetField.getType()));
+        fields.add(SqlUtil.deriveAliasFromOrdinal(fields.size()),
+            targetField.getType());
       }
     }
     final Set<Integer> assignedFields = new HashSet<>();
@@ -5699,13 +5699,13 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
     }
 
-    List<Map.Entry<String, RelDataType>> measureColumns =
+    PairList<String, RelDataType> measureColumns =
         validateMeasure(matchRecognize, scope, allRows);
-    for (Map.Entry<String, RelDataType> c : measureColumns) {
-      if (!typeBuilder.nameExists(c.getKey())) {
-        typeBuilder.add(c.getKey(), c.getValue());
+    measureColumns.forEach((name, type) -> {
+      if (!typeBuilder.nameExists(name)) {
+        typeBuilder.add(name, type);
       }
-    }
+    });
 
     final RelDataType rowType = typeBuilder.build();
     if (matchRecognize.getMeasureList().size() == 0) {
@@ -5715,12 +5715,12 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
   }
 
-  private List<Map.Entry<String, RelDataType>> validateMeasure(SqlMatchRecognize mr,
+  private PairList<String, RelDataType> validateMeasure(SqlMatchRecognize mr,
       MatchRecognizeScope scope, boolean allRows) {
     final List<String> aliases = new ArrayList<>();
     final List<SqlNode> sqlNodes = new ArrayList<>();
     final SqlNodeList measures = mr.getMeasureList();
-    final List<Map.Entry<String, RelDataType>> fields = new ArrayList<>();
+    final PairList<String, RelDataType> fields = PairList.of();
 
     for (SqlNode measure : measures) {
       assert measure instanceof SqlCall;
@@ -5735,7 +5735,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final RelDataType type = deriveType(scope, expand);
       setValidatedNodeType(measure, type);
 
-      fields.add(Pair.of(alias, type));
+      fields.add(alias, type);
       sqlNodes.add(
           SqlStdOperatorTable.AS.createCall(SqlParserPos.ZERO, expand,
               new SqlIdentifier(alias, SqlParserPos.ZERO)));
@@ -5841,11 +5841,11 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     // an aggregate or as an axis.
 
     // Aggregates, e.g. "PIVOT (sum(x) AS sum_x, count(*) AS c)"
-    final List<Pair<@Nullable String, RelDataType>> aggNames = new ArrayList<>();
+    final PairList<@Nullable String, RelDataType> aggNames = PairList.of();
     pivot.forEachAgg((alias, call) -> {
       call.validate(this, scope);
       final RelDataType type = deriveType(scope, call);
-      aggNames.add(Pair.of(alias, type));
+      aggNames.add(alias, type);
       if (!(call instanceof SqlCall)
           || !(((SqlCall) call).getOperator() instanceof SqlAggFunction)) {
         throw newValidationError(call, RESOURCE.pivotAggMalformed());
@@ -5890,7 +5890,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
                     subNode.getParserPosition(), identifier, subNode)),
             true);
       });
-      Pair.forEach(aggNames, (aggAlias, aggType) ->
+      aggNames.forEach((aggAlias, aggType) ->
           typeBuilder.add(aggAlias == null ? alias : alias + "_" + aggAlias,
               aggType));
     });
@@ -5954,7 +5954,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     columnNames.addAll(unusedColumnNames);
 
     // Gather the name and type of each measure.
-    final List<Pair<String, RelDataType>> measureNameTypes = new ArrayList<>();
+    final PairList<String, RelDataType> measureNameTypes = PairList.of();
     Ord.forEach(unpivot.measureList, (measure, i) -> {
       final String measureName = ((SqlIdentifier) measure).getSimple();
       final List<RelDataType> types = new ArrayList<>();
@@ -5977,7 +5977,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
         throw newValidationError(measure,
             RESOURCE.unpivotDuplicate(measureName));
       }
-      measureNameTypes.add(Pair.of(measureName, type));
+      measureNameTypes.add(measureName, type);
     });
 
     // Gather the name and type of each axis.
@@ -5991,7 +5991,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     // The type of 'job' is derived as the least restrictive type of the values
     // ('CLERK', 'ANALYST'), namely VARCHAR(7). The derived type of 'deptno' is
     // the type of values (10, 20), namely INTEGER.
-    final List<Pair<String, RelDataType>> axisNameTypes = new ArrayList<>();
+    final PairList<String, RelDataType> axisNameTypes = PairList.of();
     Ord.forEach(unpivot.axisList, (axis, i) -> {
       final String axisName = ((SqlIdentifier) axis).getSimple();
       final List<RelDataType> types = new ArrayList<>();
@@ -6010,7 +6010,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       if (!columnNames.add(axisName)) {
         throw newValidationError(axis, RESOURCE.unpivotDuplicate(axisName));
       }
-      axisNameTypes.add(Pair.of(axisName, type));
+      axisNameTypes.add(axisName, type);
     });
 
     // Columns that have been seen as arguments to aggregates or as axes
@@ -6834,7 +6834,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
 
     @Override public @Nullable SqlNode visit(SqlIdentifier id) {
-      final SqlNode node = expandCommonColumn(select, id, (SelectScope) getScope(), validator);
+      final SqlNode node =
+          expandCommonColumn(select, id, (SelectScope) getScope(), validator);
       if (node != id) {
         return node;
       } else {
@@ -7384,8 +7385,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
 
     /** Moves fields according to the permutation. */
-    public void permute(List<SqlNode> selectItems,
-        List<Map.Entry<String, RelDataType>> fields) {
+    void permute(List<SqlNode> selectItems,
+        PairList<String, RelDataType> fields) {
       if (trivial) {
         return;
       }
@@ -7393,8 +7394,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       final List<SqlNode> oldSelectItems = ImmutableList.copyOf(selectItems);
       selectItems.clear();
       selectItems.addAll(oldSelectItems.subList(0, offset));
-      final List<Map.Entry<String, RelDataType>> oldFields =
-          ImmutableList.copyOf(fields);
+      final PairList<String, RelDataType> oldFields = fields.immutable();
       fields.clear();
       fields.addAll(oldFields.subList(0, offset));
       for (ImmutableIntList source : sources) {
@@ -7424,7 +7424,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
                   new SqlIdentifier(name, SqlParserPos.ZERO));
           type = typeFactory.createTypeWithNullability(type2, nullable);
         }
-        fields.add(Pair.of(name, type));
+        fields.add(name, type);
         selectItems.add(selectItem);
       }
     }
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 97a34918cc..0825ec2b21 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.CustomColumnResolvingTable;
 import org.apache.calcite.schema.ExtensibleTable;
 import org.apache.calcite.schema.Table;
@@ -81,6 +82,7 @@ import java.util.Properties;
 import java.util.Set;
 import java.util.TreeSet;
 
+import static org.apache.calcite.linq4j.Nullness.castNonNullList;
 import static org.apache.calcite.sql.type.NonNullableAccessors.getCharset;
 import static org.apache.calcite.sql.type.NonNullableAccessors.getCollation;
 import static org.apache.calcite.util.Static.RESOURCE;
@@ -172,14 +174,11 @@ public class SqlValidatorUtil {
         extTable == null
             ? table.getRowType().getFieldCount()
             : extTable.getExtendedColumnOffset();
-    for (final Pair<SqlIdentifier, SqlDataTypeSpec> pair : pairs(extendedColumns)) {
-      final SqlIdentifier identifier = pair.left;
-      final SqlDataTypeSpec type = pair.right;
-      extendedFields.add(
-          new RelDataTypeFieldImpl(identifier.toString(),
-              extendedFieldOffset++,
-              type.deriveType(requireNonNull(validator, "validator"))));
-    }
+    pairs(extendedColumns).forEachIndexed((i, identifier, type) ->
+        extendedFields.add(
+            new RelDataTypeFieldImpl(identifier.toString(),
+                extendedFieldOffset + i,
+                type.deriveType(requireNonNull(validator, "validator")))));
     return extendedFields.build();
   }
 
@@ -187,9 +186,9 @@ public class SqlValidatorUtil {
    * (of the form [name0, type0, name1, type1, ...])
    * into a list of (name, type) pairs. */
   @SuppressWarnings({"unchecked", "rawtypes"})
-  private static List<Pair<SqlIdentifier, SqlDataTypeSpec>> pairs(
+  private static PairList<SqlIdentifier, SqlDataTypeSpec> pairs(
       SqlNodeList extendedColumns) {
-    return Util.pairs((List) extendedColumns);
+    return PairList.backedBy((List) extendedColumns.getList());
   }
 
   /**
@@ -431,7 +430,7 @@ public class SqlValidatorUtil {
    * @return List of unique strings
    */
   @Deprecated // to be removed before 2.0
-  public static List<String> uniquify(List<String> nameList) {
+  public static List<String> uniquify(List<? extends @Nullable String> nameList) {
     return uniquify(nameList, EXPR_SUGGESTER, true);
   }
 
@@ -449,7 +448,8 @@ public class SqlValidatorUtil {
    * @return List of unique strings
    */
   @Deprecated // to be removed before 2.0
-  public static List<String> uniquify(List<String> nameList, Suggester suggester) {
+  public static List<String> uniquify(List<? extends @Nullable String> nameList,
+      Suggester suggester) {
     return uniquify(nameList, suggester, true);
   }
 
@@ -464,7 +464,7 @@ public class SqlValidatorUtil {
    *     distinct
    * @return List of unique strings
    */
-  public static List<String> uniquify(List<String> nameList,
+  public static List<String> uniquify(List<? extends @Nullable String> nameList,
       boolean caseSensitive) {
     return uniquify(nameList, EXPR_SUGGESTER, caseSensitive);
   }
@@ -485,9 +485,17 @@ public class SqlValidatorUtil {
       List<? extends @Nullable String> nameList,
       Suggester suggester,
       boolean caseSensitive) {
-    final Set<String> used = caseSensitive
-        ? new LinkedHashSet<>()
-        : new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+    final Set<String> used;
+    if (caseSensitive) {
+      // Shortcut (avoiding creating a hash map) if the list is short and has
+      // no nulls.
+      if (Util.isDefinitelyDistinctAndNonNull(nameList)) {
+        return castNonNullList(nameList);
+      }
+      used = new LinkedHashSet<>();
+    } else {
+      used = new TreeSet<>(String.CASE_INSENSITIVE_ORDER);
+    }
     int changeCount = 0;
     final List<String> newNameList = new ArrayList<>();
     for (String name : nameList) {
@@ -498,7 +506,7 @@ public class SqlValidatorUtil {
       newNameList.add(uniqueName);
     }
     return changeCount == 0
-        ? (List<String>) nameList
+        ? castNonNullList(nameList)
         : newNameList;
   }
 
@@ -523,7 +531,7 @@ public class SqlValidatorUtil {
       RelDataTypeFactory typeFactory,
       @Nullable List<String> fieldNameList,
       List<RelDataTypeField> systemFieldList) {
-    assert systemFieldList != null;
+    requireNonNull(systemFieldList, "systemFieldList");
     switch (joinType) {
     case LEFT:
       rightType =
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java b/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
index 939a3c74c1..7e442fc9f5 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/implicit/AbstractTypeCoercion.java
@@ -22,6 +22,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFactoryImpl;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlCharStringLiteral;
 import org.apache.calcite.sql.SqlCollation;
@@ -52,7 +53,6 @@ import org.checkerframework.checker.nullness.qual.Nullable;
 import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Map;
 
 import static org.apache.calcite.sql.type.NonNullableAccessors.getCollation;
 
@@ -337,13 +337,13 @@ public abstract class AbstractTypeCoercion implements TypeCoercion {
     assert rowType.isStruct();
     assert columnIndex < rowType.getFieldList().size();
 
-    final List<Map.Entry<String, RelDataType>> fieldList = new ArrayList<>();
+    final PairList<String, RelDataType> fieldList = PairList.of();
     for (int i = 0; i < rowType.getFieldCount(); i++) {
       final RelDataTypeField field = rowType.getFieldList().get(i);
       final String name = field.getName();
       final RelDataType type = field.getType();
       final RelDataType targetType = i == columnIndex ? desiredType : type;
-      fieldList.add(Pair.of(name, targetType));
+      fieldList.add(name, targetType);
     }
     updateInferredType(query, factory.createStructType(fieldList));
   }
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
index af342d0f7a..49de474cec 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelDecorrelator.java
@@ -72,6 +72,7 @@ import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlExplainFormat;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.SqlFunction;
@@ -92,7 +93,6 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 import org.apache.calcite.util.trace.CalciteTrace;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSortedMap;
@@ -525,7 +525,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     // Project projects the original expressions,
     // plus any correlated variables the input wants to pass along.
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    final PairList<RexNode, String> projects = PairList.of();
 
     List<RelDataTypeField> newInputOutput =
         newInput.getRowType().getFieldList();
@@ -547,7 +547,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       // add mapping of group keys.
       outputMap.put(idx, newPos);
       int newInputPos = requireNonNull(frame.oldToNewOutputs.get(idx));
-      projects.add(RexInputRef.of2(newInputPos, newInputOutput));
+      RexInputRef.add2(projects, newInputPos, newInputOutput);
       mapNewInputToProjOutputs.put(newInputPos, newPos);
       newPos++;
     }
@@ -560,7 +560,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
       // Now add the corVars from the input, starting from
       // position oldGroupKeyCount.
       for (Map.Entry<CorDef, Integer> entry : frame.corDefOutputs.entrySet()) {
-        projects.add(RexInputRef.of2(entry.getValue(), newInputOutput));
+        RexInputRef.add2(projects, entry.getValue(), newInputOutput);
 
         corDefOutputs.put(entry.getKey(), newPos);
         mapNewInputToProjOutputs.put(entry.getValue(), newPos);
@@ -572,7 +572,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     final int newGroupKeyCount = newPos;
     for (int i = 0; i < newInputOutput.size(); i++) {
       if (!mapNewInputToProjOutputs.containsKey(i)) {
-        projects.add(RexInputRef.of2(i, newInputOutput));
+        RexInputRef.add2(projects, i, newInputOutput);
         mapNewInputToProjOutputs.put(i, newPos);
         newPos++;
       }
@@ -583,7 +583,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     // This Project will be what the old input maps to,
     // replacing any previous mapping from old input).
     RelNode newProject = relBuilder.push(newInput)
-        .projectNamed(Pair.left(projects), Pair.right(projects), true)
+        .projectNamed(projects.leftList(), projects.rightList(), true)
         .build();
 
     // update mappings:
@@ -756,7 +756,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
     // Project projects the original expressions,
     // plus any correlated variables the input wants to pass along.
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    final PairList<RexNode, String> projects = PairList.of();
 
     // If this Project has correlated reference, create value generator
     // and produce the correlated variables in the new output.
@@ -768,27 +768,24 @@ public class RelDecorrelator implements ReflectiveVisitor {
     final Map<Integer, Integer> mapOldToNewOutputs = new HashMap<>();
     int newPos;
     for (newPos = 0; newPos < oldProjects.size(); newPos++) {
-      projects.add(
-          newPos,
-          Pair.of(
-              decorrelateExpr(requireNonNull(currentRel, "currentRel"),
-                  map, cm, oldProjects.get(newPos)),
-              relOutput.get(newPos).getName()));
+      projects.add(newPos,
+          decorrelateExpr(requireNonNull(currentRel, "currentRel"),
+              map, cm, oldProjects.get(newPos)),
+          relOutput.get(newPos).getName());
       mapOldToNewOutputs.put(newPos, newPos);
     }
 
     // Project any correlated variables the input wants to pass along.
     final NavigableMap<CorDef, Integer> corDefOutputs = new TreeMap<>();
     for (Map.Entry<CorDef, Integer> entry : frame.corDefOutputs.entrySet()) {
-      projects.add(
-          RexInputRef.of2(entry.getValue(),
-              frame.r.getRowType().getFieldList()));
+      RexInputRef.add2(projects, entry.getValue(),
+          frame.r.getRowType().getFieldList());
       corDefOutputs.put(entry.getKey(), newPos);
       newPos++;
     }
 
     RelNode newProject = relBuilder.push(frame.r)
-        .projectNamed(Pair.left(projects), Pair.right(projects), true)
+        .projectNamed(projects.leftList(), projects.rightList(), true)
         .build();
 
     return register(rel, newProject, mapOldToNewOutputs, corDefOutputs);
@@ -1414,7 +1411,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
                 true));
 
     // now create the new project
-    List<Pair<RexNode, String>> newProjExprs = new ArrayList<>();
+    final PairList<RexNode, String> newProjExprs = PairList.of();
 
     // project everything from the LHS and then those from the original
     // projRel
@@ -1422,7 +1419,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         left.getRowType().getFieldList();
 
     for (int i = 0; i < leftInputFields.size(); i++) {
-      newProjExprs.add(RexInputRef.of2(i, leftInputFields));
+      RexInputRef.add2(newProjExprs, i, leftInputFields);
     }
 
     // Marked where the projected expr is coming from so that the types will
@@ -1438,11 +1435,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
               projectPulledAboveLeftCorrelator,
               nullIndicator);
 
-      newProjExprs.add(Pair.of(newProjExpr, pair.right));
+      newProjExprs.add(newProjExpr, pair.right);
     }
 
     return relBuilder.push(join)
-        .projectNamed(Pair.left(newProjExprs), Pair.right(newProjExprs), true)
+        .projectNamed(newProjExprs.leftList(), newProjExprs.rightList(), true)
         .build();
   }
 
@@ -1464,7 +1461,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
     final JoinRelType joinType = correlate.getJoinType();
 
     // now create the new project
-    final List<Pair<RexNode, String>> newProjects = new ArrayList<>();
+    final PairList<RexNode, String> newProjects = PairList.of();
 
     // Project everything from the LHS and then those from the original
     // project
@@ -1472,7 +1469,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         left.getRowType().getFieldList();
 
     for (int i = 0; i < leftInputFields.size(); i++) {
-      newProjects.add(RexInputRef.of2(i, leftInputFields));
+      RexInputRef.add2(newProjects, i, leftInputFields);
     }
 
     // Marked where the projected expr is coming from so that the types will
@@ -1487,11 +1484,11 @@ public class RelDecorrelator implements ReflectiveVisitor {
               pair.left,
               projectPulledAboveLeftCorrelator,
               isCount);
-      newProjects.add(Pair.of(newProjExpr, pair.right));
+      newProjects.add(newProjExpr, pair.right);
     }
 
     return relBuilder.push(correlate)
-        .projectNamed(Pair.left(newProjects), Pair.right(newProjects), true)
+        .projectNamed(newProjects.leftList(), newProjects.rightList(), true)
         .build();
   }
 
@@ -1569,17 +1566,17 @@ public class RelDecorrelator implements ReflectiveVisitor {
    */
   private RelNode createProjectWithAdditionalExprs(
       RelNode input,
-      List<Pair<RexNode, String>> additionalExprs) {
+      PairList<RexNode, String> additionalExprs) {
     final List<RelDataTypeField> fieldList =
         input.getRowType().getFieldList();
-    List<Pair<RexNode, String>> projects = new ArrayList<>();
+    PairList<RexNode, String> projects = PairList.of();
     Ord.forEach(fieldList, (field, i) ->
         projects.add(
-            Pair.of(relBuilder.getRexBuilder().makeInputRef(field.getType(), i),
-                field.getName())));
+            relBuilder.getRexBuilder().makeInputRef(field.getType(), i),
+            field.getName()));
     projects.addAll(additionalExprs);
     return relBuilder.push(input)
-        .projectNamed(Pair.left(projects), Pair.right(projects), true)
+        .projectNamed(projects.leftList(), projects.rightList(), true)
         .build();
   }
 
@@ -2124,8 +2121,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
         // make the new Project to provide a null indicator
         right =
             d.createProjectWithAdditionalExprs(right,
-                ImmutableList.of(
-                    Pair.of(d.relBuilder.literal(true), "nullIndicator")));
+                PairList.of(d.relBuilder.literal(true), "nullIndicator"));
 
         // make the new aggRel
         right =
@@ -2440,8 +2436,7 @@ public class RelDecorrelator implements ReflectiveVisitor {
 
       right =
           d.createProjectWithAdditionalExprs(right,
-              ImmutableList.of(
-                  Pair.of(rexBuilder.makeLiteral(true), "nullIndicator")));
+              PairList.of(rexBuilder.makeLiteral(true), "nullIndicator"));
 
       Join join =
           (Join) d.relBuilder
@@ -2601,15 +2596,15 @@ public class RelDecorrelator implements ReflectiveVisitor {
         aggregate = call.rel(2);
 
         // Create identity projection
-        final List<Pair<RexNode, String>> projects = new ArrayList<>();
+        final PairList<RexNode, String> projects = PairList.of();
         final List<RelDataTypeField> fields =
             aggregate.getRowType().getFieldList();
         for (int i = 0; i < fields.size(); i++) {
-          projects.add(RexInputRef.of2(projects.size(), fields));
+          RexInputRef.add2(projects, projects.size(), fields);
         }
         final RelBuilder relBuilder = call.builder();
         relBuilder.push(aggregate)
-            .projectNamed(Pair.left(projects), Pair.right(projects), true);
+            .projectNamed(projects.leftList(), projects.rightList(), true);
         aggOutputProject = (Project) relBuilder.build();
       }
       onMatch2(call, correlate, left, aggOutputProject, aggregate);
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index 1d0570eb7a..2ad2e35d0b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -66,6 +66,7 @@ import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
@@ -73,7 +74,6 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
-import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectUtil;
 import org.apache.calcite.util.ReflectiveVisitDispatcher;
 import org.apache.calcite.util.ReflectiveVisitor;
@@ -541,13 +541,12 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     RewriteRexShuttle shuttle = new RewriteRexShuttle();
     List<RexNode> oldProjects = rel.getProjects();
     List<String> oldNames = rel.getRowType().getFieldNames();
-    List<Pair<RexNode, String>> flattenedExpList = new ArrayList<>();
+    PairList<RexNode, String> flattenedExpList = PairList.of();
     flattenProjections(shuttle, oldProjects, oldNames, "", flattenedExpList);
     RelNode newInput = getNewForOldRel(rel.getInput());
-    List<RexNode> newProjects = Pair.left(flattenedExpList);
-    List<String> newNames = Pair.right(flattenedExpList);
     final RelNode newRel = relBuilder.push(newInput)
-        .projectNamed(newProjects, newNames, true)
+        .projectNamed(flattenedExpList.leftList(), flattenedExpList.rightList(),
+            true)
         .hints(rel.getHints())
         .build();
     setNewForOldRel(rel, newRel);
@@ -571,7 +570,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     }
 
     // Convert the projections.
-    final List<Pair<RexNode, String>> flattenedExpList = new ArrayList<>();
+    final PairList<RexNode, String> flattenedExpList = PairList.of();
     List<String> fieldNames = rel.getRowType().getFieldNames();
     flattenProjections(new RewriteRexShuttle(),
         program.getProjectList(),
@@ -580,9 +579,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         flattenedExpList);
 
     // Register each of the new projections.
-    for (Pair<RexNode, String> flattenedExp : flattenedExpList) {
-      programBuilder.addProject(flattenedExp.left, flattenedExp.right);
-    }
+    flattenedExpList.forEach(programBuilder::addProject);
 
     // Translate the condition.
     final RexLocalRef conditionRef = program.getCondition();
@@ -626,7 +623,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       List<? extends RexNode> exps,
       @Nullable List<? extends @Nullable String> fieldNames,
       String prefix,
-      List<Pair<RexNode, String>> flattenedExps) {
+      PairList<RexNode, String> flattenedExps) {
     for (int i = 0; i < exps.size(); ++i) {
       RexNode exp = exps.get(i);
       String fieldName = extractName(fieldNames, prefix, i);
@@ -648,7 +645,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   private void flattenProjection(RewriteRexShuttle shuttle,
       RexNode exp,
       String fieldName,
-      List<Pair<RexNode, String>> flattenedExps) {
+      PairList<RexNode, String> flattenedExps) {
     if (exp.getType().isStruct()) {
       if (exp instanceof RexInputRef) {
         final int oldOrdinal = ((RexInputRef) exp).getIndex();
@@ -656,7 +653,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         for (int innerOrdinal = 0; innerOrdinal < flattenFieldsCount; innerOrdinal++) {
           Ord<RelDataType> newField = getNewFieldForOldInput(oldOrdinal, innerOrdinal);
           RexInputRef newRef = new RexInputRef(newField.i, newField.e);
-          flattenedExps.add(Pair.of(newRef, fieldName));
+          flattenedExps.add(newRef, fieldName);
         }
       } else if (isConstructor(exp) || exp.isA(SqlKind.CAST)) {
         // REVIEW jvs 27-Feb-2005:  for cast, see corresponding note
@@ -699,9 +696,11 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
                 int oldOrdinal = ((RexInputRef) firstOp).getIndex();
                 int to = from + postFlattenSize(field.getType());
                 for (int newInnerOrdinal = from; newInnerOrdinal < to; newInnerOrdinal++) {
-                  Ord<RelDataType> newField = getNewFieldForOldInput(oldOrdinal, newInnerOrdinal);
-                  RexInputRef newRef = rexBuilder.makeInputRef(newField.e, newField.i);
-                  flattenedExps.add(Pair.of(newRef, fieldName));
+                  Ord<RelDataType> newField =
+                      getNewFieldForOldInput(oldOrdinal, newInnerOrdinal);
+                  RexInputRef newRef =
+                      rexBuilder.makeInputRef(newField.e, newField.i);
+                  flattenedExps.add(newRef, fieldName);
                 }
                 break;
               } else {
@@ -712,8 +711,9 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
             // to get nested struct from return type of firstOp rex call,
             // we need to flatten firstOp and get range of expressions which
             // corresponding to desirable nested struct flattened fields
-            List<Pair<RexNode, String>> firstOpFlattenedExps = new ArrayList<>();
-            flattenProjection(shuttle, firstOp, fieldName + "$0", firstOpFlattenedExps);
+            PairList<RexNode, String> firstOpFlattenedExps = PairList.of();
+            flattenProjection(shuttle, firstOp, fieldName + "$0",
+                firstOpFlattenedExps);
             int newInnerOrdinal = getNewInnerOrdinal(firstOp, literalString);
             int endOfRange = newInnerOrdinal + postFlattenSize(newExp.getType());
             for (int i = newInnerOrdinal; i < endOfRange; i++) {
@@ -731,14 +731,13 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         throw Util.needToImplement(exp);
       }
     } else {
-      flattenedExps.add(
-          Pair.of(exp.accept(shuttle), fieldName));
+      flattenedExps.add(exp.accept(shuttle), fieldName);
     }
   }
 
   private void flattenResultTypeOfRexCall(RexNode newExp,
       String fieldName,
-      List<Pair<RexNode, String>> flattenedExps) {
+      PairList<RexNode, String> flattenedExps) {
     int nameIdx = 0;
     for (RelDataTypeField field : newExp.getType().getFieldList()) {
       RexNode fieldRef = rexBuilder.makeFieldAccess(newExp, field.getIndex());
@@ -746,21 +745,18 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       if (fieldRef.getType().isStruct()) {
         flattenResultTypeOfRexCall(fieldRef, fieldRefName, flattenedExps);
       } else {
-        flattenedExps.add(Pair.of(fieldRef, fieldRefName));
+        flattenedExps.add(fieldRef, fieldRefName);
       }
     }
   }
 
-  private void flattenNullLiteral(
-      RelDataType type,
-      List<Pair<RexNode, String>> flattenedExps) {
+  private void flattenNullLiteral(RelDataType type,
+      PairList<RexNode, String> flattenedExps) {
     RelDataType flattenedType =
         SqlTypeUtil.flattenRecordType(rexBuilder.getTypeFactory(), type, null);
     for (RelDataTypeField field : flattenedType.getFieldList()) {
-      flattenedExps.add(
-          Pair.of(
-              rexBuilder.makeNullLiteral(field.getType()),
-              field.getName()));
+      flattenedExps.add(rexBuilder.makeNullLiteral(field.getType()),
+          field.getName());
     }
   }
 
@@ -783,17 +779,15 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   }
 
   private RelNode coverNewRelByFlatteningProjection(RelNode rel, RelNode newRel) {
-    final List<Pair<RexNode, String>> flattenedExpList = new ArrayList<>();
+    final PairList<RexNode, String> flattenedExpList = PairList.of();
     RexNode newRowRef = rexBuilder.makeRangeReference(newRel);
     List<RelDataTypeField> inputRowFields = rel.getRowType().getFieldList();
     flattenInputs(inputRowFields, newRowRef, flattenedExpList);
     // cover new scan with flattening projection
-    List<RexNode> projects = Pair.left(flattenedExpList);
-    List<String> fieldNames = Pair.right(flattenedExpList);
-    newRel = relBuilder.push(newRel)
-        .projectNamed(projects, fieldNames, true)
+    return relBuilder.push(newRel)
+        .projectNamed(flattenedExpList.leftList(), flattenedExpList.rightList(),
+            true)
         .build();
-    return newRel;
   }
 
   public void rewriteRel(LogicalSnapshot rel) {
@@ -819,7 +813,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
   /** Generates expressions that reference the flattened input fields from
    * a given row type. */
   private void flattenInputs(List<RelDataTypeField> fieldList, RexNode prefix,
-      List<Pair<RexNode, String>> flattenedExpList) {
+      PairList<RexNode, String> flattenedExpList) {
     for (RelDataTypeField field : fieldList) {
       final RexNode ref =
           rexBuilder.makeFieldAccess(prefix, field.getIndex());
@@ -827,7 +821,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         final List<RelDataTypeField> structFields = field.getType().getFieldList();
         flattenInputs(structFields, ref, flattenedExpList);
       } else {
-        flattenedExpList.add(Pair.of(ref, field.getName()));
+        flattenedExpList.add(ref, field.getName());
       }
     }
   }
@@ -1024,7 +1018,7 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
         RexBuilder rexBuilder,
         SqlOperator op,
         @MinLen(1) List<RexNode> exprs) {
-      final List<Pair<RexNode, String>> flattenedExps = new ArrayList<>();
+      final PairList<RexNode, String> flattenedExps = PairList.of();
       flattenProjections(this, exprs, null, "", flattenedExps);
       int n = flattenedExps.size() / 2;
       if (n == 0) {
@@ -1041,31 +1035,24 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
       }
       RexNode conjunction = null;
       for (int i = 0; i < n; ++i) {
+        final List<RexNode> leftList = flattenedExps.leftList();
         RexNode comparison =
-            rexBuilder.makeCall(
-                op,
-                flattenedExps.get(i).left,
-                flattenedExps.get(i + n).left);
+            rexBuilder.makeCall(op, leftList.get(i), leftList.get(i + n));
         if (conjunction == null) {
           conjunction = comparison;
         } else {
           conjunction =
-              rexBuilder.makeCall(
-                  SqlStdOperatorTable.AND,
-                  conjunction,
+              rexBuilder.makeCall(SqlStdOperatorTable.AND, conjunction,
                   comparison);
         }
       }
       requireNonNull(conjunction, "conjunction must be non-null");
       if (negate) {
-        return rexBuilder.makeCall(
-            SqlStdOperatorTable.NOT,
-            conjunction);
+        return rexBuilder.makeCall(SqlStdOperatorTable.NOT, conjunction);
       } else {
         return conjunction;
       }
     }
-
   }
 
   private int getNewInnerOrdinal(RexNode firstOp, @Nullable String literalString) {
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index 97c746aeda..9f86720327 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -94,6 +94,7 @@ import org.apache.calcite.rex.RexSubQuery;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.rex.RexWindowBounds;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.ColumnStrategy;
 import org.apache.calcite.schema.ModifiableTable;
 import org.apache.calcite.schema.ModifiableView;
@@ -193,7 +194,6 @@ import org.slf4j.Logger;
 
 import java.lang.reflect.Type;
 import java.math.BigDecimal;
-import java.util.AbstractList;
 import java.util.ArrayDeque;
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -208,6 +208,7 @@ import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
 import java.util.function.Supplier;
 import java.util.function.UnaryOperator;
@@ -835,37 +836,37 @@ public class SqlToRelConverter {
 
       final Map<Integer, Integer> squished = new HashMap<>();
       final List<RelDataTypeField> fields = rel.getRowType().getFieldList();
-      final List<Pair<RexNode, String>> newProjects = new ArrayList<>();
+      final PairList<RexNode, String> newProjects = PairList.of();
       for (int i = 0; i < fields.size(); i++) {
         if (origins.get(i) == i) {
           squished.put(i, newProjects.size());
-          newProjects.add(RexInputRef.of2(i, fields));
+          RexInputRef.add2(newProjects, i, fields);
         }
       }
       rel =
           LogicalProject.create(rel, ImmutableList.of(),
-              Pair.left(newProjects), Pair.right(newProjects), project.getVariablesSet());
+              newProjects.leftList(), newProjects.rightList(),
+              project.getVariablesSet());
       bb.root = rel;
       distinctify(bb, false);
       rel = bb.root();
 
       // Create the expressions to reverse the mapping.
       // Project($0, $1, $0, $2).
-      final List<Pair<RexNode, String>> undoProjects = new ArrayList<>();
+      final PairList<RexNode, String> undoProjects = PairList.of();
       for (int i = 0; i < fields.size(); i++) {
         final int origin = origins.get(i);
         RelDataTypeField field = fields.get(i);
         undoProjects.add(
-            Pair.of(
-                new RexInputRef(
-                    castNonNull(squished.get(origin)),
-                    field.getType()),
-                field.getName()));
+            new RexInputRef(castNonNull(squished.get(origin)),
+                field.getType()),
+            field.getName());
       }
 
       rel =
           LogicalProject.create(rel, ImmutableList.of(),
-              Pair.left(undoProjects), Pair.right(undoProjects), ImmutableSet.of());
+              undoProjects.leftList(), undoProjects.rightList(),
+              ImmutableSet.of());
       bb.setRoot(
           rel,
           false);
@@ -2664,9 +2665,10 @@ public class SqlToRelConverter {
     pivotBb.agg = null;
 
     // Project the fields that we will need.
+    final PairList<RexNode, @Nullable String> pairs =
+        aggConverter.getPreExprs();
     relBuilder
-        .project(Pair.left(aggConverter.getPreExprs()),
-            Pair.right(aggConverter.getPreExprs()));
+        .project(pairs.leftList(), pairs.rightList());
 
     // Build expressions.
 
@@ -3398,7 +3400,7 @@ public class SqlToRelConverter {
     }
 
     final RexNode havingExpr;
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    final PairList<RexNode, String> projects = PairList.of();
 
     try {
       checkArgument(bb.agg == null, "already in agg mode");
@@ -3420,14 +3422,14 @@ public class SqlToRelConverter {
       }
 
       // compute inputs to the aggregator
-      List<Pair<RexNode, @Nullable String>> preExprs = aggConverter.getPreExprs();
+      PairList<RexNode, @Nullable String> preExprs = aggConverter.getPreExprs();
 
       if (preExprs.size() == 0) {
         // Special case for COUNT(*), where we can end up with no inputs
         // at all.  The rest of the system doesn't like 0-tuples, so we
         // select a dummy constant here.
         final RexNode zero = rexBuilder.makeExactLiteral(BigDecimal.ZERO);
-        preExprs = ImmutableList.of(Pair.of(zero, null));
+        preExprs = PairList.of(zero, null);
       }
 
       final RelNode inputRel = bb.root();
@@ -3435,7 +3437,7 @@ public class SqlToRelConverter {
       // Project the expressions required by agg and having.
       bb.setRoot(
           relBuilder.push(inputRel)
-              .projectNamed(Pair.left(preExprs), Pair.right(preExprs), false)
+              .projectNamed(preExprs.leftList(), preExprs.rightList(), false)
               .build(),
           false);
       bb.mapRootRelToFieldProjection.put(bb.root(), r.groupExprProjection);
@@ -3494,17 +3496,15 @@ public class SqlToRelConverter {
           selectNamespace.getRowType().getFieldNames();
       int sysFieldCount = selectList.size() - names.size();
       for (SqlNode expr : selectList) {
-        projects.add(
-            Pair.of(bb.convertExpression(expr),
-                k < sysFieldCount
-                    ? SqlValidatorUtil.alias(expr, k++)
-                    : names.get(k++ - sysFieldCount)));
+        projects.add(bb.convertExpression(expr),
+            k < sysFieldCount
+                ? SqlValidatorUtil.alias(expr, k++)
+                : names.get(k++ - sysFieldCount));
       }
 
       for (SqlNode expr : orderExprList) {
-        projects.add(
-            Pair.of(bb.convertExpression(expr),
-                SqlValidatorUtil.alias(expr, k++)));
+        projects.add(bb.convertExpression(expr),
+            SqlValidatorUtil.alias(expr, k++));
       }
     } finally {
       bb.agg = null;
@@ -3515,8 +3515,8 @@ public class SqlToRelConverter {
     relBuilder.filter(havingExpr);
 
     // implement the SELECT list
-    relBuilder.project(Pair.left(projects), Pair.right(projects))
-        .rename(Pair.right(projects));
+    relBuilder.project(projects.leftList(), projects.rightList())
+        .rename(projects.rightList());
     bb.setRoot(relBuilder.build(), false);
 
     // Tell bb which of group columns are sorted.
@@ -3859,19 +3859,18 @@ public class SqlToRelConverter {
     final RexNode constraint =
         modifiableView.getConstraint(rexBuilder, delegateRowType);
     RelOptUtil.inferViewPredicates(projectMap, filters, constraint);
-    final List<Pair<RexNode, String>> projects = new ArrayList<>();
+    final PairList<RexNode, String> projects = PairList.of();
     for (RelDataTypeField field : delegateRowType.getFieldList()) {
       RexNode node = projectMap.get(field.getIndex());
       if (node == null) {
         node = rexBuilder.makeNullLiteral(field.getType());
       }
-      projects.add(
-          Pair.of(rexBuilder.ensureType(field.getType(), node, false),
-              field.getName()));
+      projects.add(rexBuilder.ensureType(field.getType(), node, false),
+          field.getName());
     }
 
     return relBuilder.push(source)
-        .projectNamed(Pair.left(projects), Pair.right(projects), false)
+        .projectNamed(projects.leftList(), projects.rightList(), false)
         .filter(filters)
         .build();
   }
@@ -4764,17 +4763,16 @@ public class SqlToRelConverter {
       Blackboard tmpBb = createBlackboard(bb.scope, null, false);
       replaceSubQueries(tmpBb, rowConstructor,
           RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
-      final List<Pair<RexNode, String>> exps = new ArrayList<>();
+      final PairList<RexNode, String> exps = PairList.of();
       Ord.forEach(rowConstructor.getOperandList(), (operand, i) ->
-          exps.add(
-              Pair.of(tmpBb.convertExpression(operand),
-                  SqlValidatorUtil.alias(operand, i))));
+          exps.add(tmpBb.convertExpression(operand),
+              SqlValidatorUtil.alias(operand, i)));
       RelNode in =
           (null == tmpBb.root)
               ? LogicalValues.createOneRow(cluster)
               : tmpBb.root;
       relBuilder.push(in)
-          .project(Pair.left(exps), Pair.right(exps));
+          .project(exps.leftList(), exps.rightList());
     }
 
     bb.setRoot(
@@ -4995,22 +4993,13 @@ public class SqlToRelConverter {
         assert leftKeyCount == rightFieldLength - 1;
 
         final int rexRangeRefLength = leftKeyCount + rightFieldLength;
-        RelDataType returnType =
-            typeFactory.createStructType(
-                new AbstractList<Map.Entry<String, RelDataType>>() {
-                  @Override public Map.Entry<String, RelDataType> get(
-                      int index) {
-                    return join.getRowType().getFieldList()
-                        .get(origLeftInputCount + index);
-                  }
-
-                  @Override public int size() {
-                    return rexRangeRefLength;
-                  }
-                });
+        final RelDataTypeFactory.Builder builder = typeFactory.builder();
+        for (int i = 0; i < rexRangeRefLength; i++) {
+          builder.add(join.getRowType().getFieldList()
+              .get(origLeftInputCount + i));
+        }
 
-        return rexBuilder.makeRangeReference(
-            returnType,
+        return rexBuilder.makeRangeReference(builder.build(),
             origLeftInputCount,
             false);
       } else {
@@ -5196,10 +5185,10 @@ public class SqlToRelConverter {
     RexNode lookup(
         int offset,
         LookupContext lookupContext) {
-      Pair<RelNode, Integer> pair = lookupContext.findRel(offset);
+      Map.Entry<RelNode, Integer> pair = lookupContext.findRel(offset);
       return rexBuilder.makeRangeReference(
-          pair.left.getRowType(),
-          pair.right,
+          pair.getKey().getRowType(),
+          pair.getValue(),
           false);
     }
 
@@ -5223,26 +5212,20 @@ public class SqlToRelConverter {
         List<RelNode> rels,
         int systemFieldCount,
         int[] start,
-        List<Pair<RelNode, Integer>> relOffsetList) {
+        BiConsumer<RelNode, Integer> relOffsetList) {
       for (RelNode rel : rels) {
         if (leaves.containsKey(rel)) {
-          relOffsetList.add(
-              Pair.of(rel, start[0]));
+          relOffsetList.accept(rel, start[0]);
           start[0] += leaves.get(rel);
         } else if (rel instanceof LogicalMatch) {
-          relOffsetList.add(
-              Pair.of(rel, start[0]));
+          relOffsetList.accept(rel, start[0]);
           start[0] += rel.getRowType().getFieldCount();
         } else {
           if (rel instanceof LogicalJoin
               || rel instanceof LogicalAggregate) {
             start[0] += systemFieldCount;
           }
-          flatten(
-              rel.getInputs(),
-              systemFieldCount,
-              start,
-              relOffsetList);
+          flatten(rel.getInputs(), systemFieldCount, start, relOffsetList);
         }
       }
     }
@@ -5771,8 +5754,8 @@ public class SqlToRelConverter {
      * aggregates. The right field of each pair is the name of the expression,
      * where the expressions are simple mappings to input fields.
      */
-    private final List<Pair<RexNode, @Nullable String>> convertedInputExprs =
-        new ArrayList<>();
+    private final PairList<RexNode, @Nullable String> convertedInputExprs =
+        PairList.of();
 
     /** Expressions to be evaluated as rows are being placed into the
      * aggregate's hash table. This is when group functions such as TUMBLE
@@ -5863,12 +5846,12 @@ public class SqlToRelConverter {
         final int i = ((RexInputRef) expr).getIndex();
         name = bb.root().getRowType().getFieldList().get(i).getName();
       }
-      if (Pair.right(convertedInputExprs).contains(name)) {
+      if (convertedInputExprs.rightList().contains(name)) {
         // In case like 'SELECT ... GROUP BY x, y, x', don't add
         // name 'x' twice.
         name = null;
       }
-      convertedInputExprs.add(Pair.of(expr, name));
+      convertedInputExprs.add(expr, name);
     }
 
     @Override public Void visit(SqlIdentifier id) {
@@ -6175,7 +6158,7 @@ public class SqlToRelConverter {
               groupExprs.size(),
               aggCalls,
               aggCallMapping,
-              i -> convertedInputExprs.get(i).left.getType().isNullable());
+              i -> convertedInputExprs.leftList().get(i).getType().isNullable());
       aggMapping.put(outerCall, rex);
     }
 
@@ -6192,7 +6175,7 @@ public class SqlToRelConverter {
 
     private int lookupOrCreateGroupExpr(RexNode expr) {
       int index = 0;
-      for (RexNode convertedInputExpr : Pair.left(convertedInputExprs)) {
+      for (RexNode convertedInputExpr : convertedInputExprs.leftList()) {
         if (expr.equals(convertedInputExpr)) {
           return index;
         }
@@ -6229,7 +6212,7 @@ public class SqlToRelConverter {
           AuxiliaryConverter converter = e.getValue().e;
           final int groupOrdinal = e.getValue().i;
           return converter.convert(rexBuilder,
-              convertedInputExprs.get(groupOrdinal).left,
+              convertedInputExprs.leftList().get(groupOrdinal),
               rexBuilder.makeInputRef(castNonNull(bb.root), groupOrdinal));
         }
       }
@@ -6237,7 +6220,7 @@ public class SqlToRelConverter {
       return aggMapping.get(call);
     }
 
-    public List<Pair<RexNode, @Nullable String>> getPreExprs() {
+    public PairList<RexNode, @Nullable String> getPreExprs() {
       return convertedInputExprs;
     }
 
@@ -6254,8 +6237,7 @@ public class SqlToRelConverter {
    * Context to find a relational expression to a field offset.
    */
   private static class LookupContext {
-    private final List<Pair<RelNode, Integer>> relOffsetList =
-        new ArrayList<>();
+    private final PairList<RelNode, Integer> relOffsetList = PairList.of();
 
     /**
      * Creates a LookupContext with multiple input relational expressions.
@@ -6265,7 +6247,7 @@ public class SqlToRelConverter {
      * @param systemFieldCount Number of system fields
      */
     LookupContext(Blackboard bb, List<RelNode> rels, int systemFieldCount) {
-      bb.flatten(rels, systemFieldCount, new int[]{0}, relOffsetList);
+      bb.flatten(rels, systemFieldCount, new int[]{0}, relOffsetList::add);
     }
 
     /**
@@ -6280,7 +6262,7 @@ public class SqlToRelConverter {
      * @param offset Offset of relational expression in FROM clause
      * @return Relational expression and the ordinal of its first field
      */
-    Pair<RelNode, Integer> findRel(int offset) {
+    Map.Entry<RelNode, Integer> findRel(int offset) {
       return relOffsetList.get(offset);
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 7c014d490c..1c07a10c3f 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -84,6 +84,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.rex.RexWindowBounds;
 import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.TransientTable;
 import org.apache.calcite.schema.impl.ListTransientTable;
 import org.apache.calcite.sql.SqlAggFunction;
@@ -152,7 +153,6 @@ import java.util.function.Consumer;
 import java.util.function.Function;
 import java.util.function.UnaryOperator;
 import java.util.stream.Collectors;
-import java.util.stream.StreamSupport;
 
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 import static org.apache.calcite.rel.rules.AggregateRemoveRule.canFlattenStatic;
@@ -2493,9 +2493,9 @@ public class RelBuilder {
     // There are duplicate aggregate calls. Rebuild the list to eliminate
     // duplicates, then add a Project.
     final Set<AggregateCall> callSet = new HashSet<>();
-    final List<Pair<Integer, @Nullable String>> projects = new ArrayList<>();
+    final PairList<Integer, @Nullable String> projects = PairList.of();
     Util.range(groupSet.cardinality())
-        .forEach(i -> projects.add(Pair.of(i, null)));
+        .forEach(i -> projects.add(i, null));
     final List<AggregateCall> distinctAggregateCalls = new ArrayList<>();
     for (AggregateCall aggregateCall : aggregateCalls) {
       final int i;
@@ -2506,14 +2506,11 @@ public class RelBuilder {
         i = distinctAggregateCalls.indexOf(aggregateCall);
         assert i >= 0;
       }
-      projects.add(Pair.of(groupSet.cardinality() + i, aggregateCall.name));
+      projects.add(groupSet.cardinality() + i, aggregateCall.name);
     }
     aggregate_(groupSet, groupSets, r, distinctAggregateCalls,
         registrar.extraNodes, inFields);
-    final List<RexNode> fields = projects.stream()
-        .map(p -> aliasMaybe(field(p.left), p.right))
-        .collect(Collectors.toList());
-    return project(fields);
+    return project(projects.transform((i, name) -> aliasMaybe(field(i), name)));
   }
 
   /** Returns whether an input is already unique, and therefore a Project
@@ -3747,16 +3744,13 @@ public class RelBuilder {
     // Make immutable copies of all arguments.
     final List<String> measureNameList = ImmutableList.copyOf(measureNames);
     final List<String> axisNameList = ImmutableList.copyOf(axisNames);
-    final List<Pair<List<RexLiteral>, List<RexNode>>> map =
-        StreamSupport.stream(axisMap.spliterator(), false)
-            .map(pair ->
-                Pair.<List<RexLiteral>, List<RexNode>>of(
-                    ImmutableList.<RexLiteral>copyOf(pair.getKey()),
-                    ImmutableList.<RexNode>copyOf(pair.getValue())))
-            .collect(Util.toImmutableList());
+    final PairList<List<RexLiteral>, List<RexNode>> map = PairList.of();
+    Pair.forEach(axisMap, (valueList, inputMeasureList) ->
+        map.add(ImmutableList.copyOf(valueList),
+            ImmutableList.copyOf(inputMeasureList)));
 
     // Check that counts match.
-    Pair.forEach(map, (valueList, inputMeasureList) -> {
+    map.forEach((valueList, inputMeasureList) -> {
       if (inputMeasureList.size() != measureNameList.size()) {
         throw new IllegalArgumentException("Number of measures ("
             + inputMeasureList.size() + ") must match number of measure names ("
@@ -3771,7 +3765,7 @@ public class RelBuilder {
 
     final RelDataType leftRowType = peek().getRowType();
     final BitSet usedFields = new BitSet();
-    Pair.forEach(map, (aliases, nodes) ->
+    map.forEach((aliases, nodes) ->
         nodes.forEach(node -> {
           if (node instanceof RexInputRef) {
             usedFields.set(((RexInputRef) node).getIndex());
@@ -3779,7 +3773,7 @@ public class RelBuilder {
         }));
 
     // Create "VALUES (('commission'), ('salary')) AS t (remuneration_type)"
-    values(ImmutableList.copyOf(Pair.left(map)), axisNameList);
+    values(ImmutableList.copyOf(map.leftList()), axisNameList);
 
     join(JoinRelType.INNER);
 
@@ -3795,7 +3789,7 @@ public class RelBuilder {
     final List<RexNode> conditions = new ArrayList<>();
     Ord.forEach(measureNameList, (measureName, m) -> {
       final List<RexNode> caseOperands = new ArrayList<>();
-      Pair.forEach(map, (literals, nodes) -> {
+      map.forEach((literals, nodes) -> {
         Ord.forEach(literals, (literal, d) ->
             conditions.add(
                 equals(field(leftRowType.getFieldCount() + d), literal)));
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 4e50e002f8..a0826d1a5d 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -89,6 +89,7 @@ import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.runtime.FunctionContexts;
 import org.apache.calcite.runtime.JsonFunctions;
 import org.apache.calcite.runtime.Matcher;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.runtime.Pattern;
 import org.apache.calcite.runtime.RandomFunction;
 import org.apache.calcite.runtime.ResultSetEnumerable;
@@ -263,6 +264,8 @@ public enum BuiltInMethod {
   FUNCTION1_APPLY(Function1.class, "apply", Object.class),
   ARRAYS_AS_LIST(Arrays.class, "asList", Object[].class),
   ARRAY(SqlFunctions.class, "array", Object[].class),
+  PAIR_LIST_COPY_OF(PairList.class, "copyOf", Object.class, Object.class,
+      Object[].class),
   FLAT_PRODUCT(SqlFunctions.class, "flatProduct", int[].class, boolean.class,
       FlatProductInputType[].class),
   FLAT_LIST(SqlFunctions.class, "flatList"),
diff --git a/core/src/main/java/org/apache/calcite/util/Pair.java b/core/src/main/java/org/apache/calcite/util/Pair.java
index 1366018840..342e9310f7 100644
--- a/core/src/main/java/org/apache/calcite/util/Pair.java
+++ b/core/src/main/java/org/apache/calcite/util/Pair.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.util;
 
+import org.apache.calcite.runtime.PairList;
+
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.io.Serializable;
@@ -91,9 +93,9 @@ public class Pair<T1 extends @Nullable Object, T2 extends @Nullable Object>
 
   @Override public boolean equals(@Nullable Object obj) {
     return this == obj
-        || (obj instanceof Pair)
-        && Objects.equals(this.left, ((Pair) obj).left)
-        && Objects.equals(this.right, ((Pair) obj).right);
+        || (obj instanceof Map.Entry)
+        && Objects.equals(this.left, ((Map.Entry) obj).getKey())
+        && Objects.equals(this.right, ((Map.Entry) obj).getValue());
   }
 
   /** {@inheritDoc}
@@ -323,13 +325,21 @@ public class Pair<T1 extends @Nullable Object, T2 extends @Nullable Object>
     return Util.transform(iterable, Map.Entry::getValue);
   }
 
+  @SuppressWarnings("unchecked")
   public static <K, V> List<K> left(
       final List<? extends Map.Entry<? extends K, ? extends V>> pairs) {
+    if (pairs instanceof PairList) {
+      return ((PairList<K, V>) pairs).leftList();
+    }
     return Util.transform(pairs, Map.Entry::getKey);
   }
 
+  @SuppressWarnings("unchecked")
   public static <K, V> List<V> right(
       final List<? extends Map.Entry<? extends K, ? extends V>> pairs) {
+    if (pairs instanceof PairList) {
+      return ((PairList<K, V>) pairs).rightList();
+    }
     return Util.transform(pairs, Map.Entry::getValue);
   }
 
diff --git a/core/src/main/java/org/apache/calcite/util/Util.java b/core/src/main/java/org/apache/calcite/util/Util.java
index 66dae20104..174779b8b2 100644
--- a/core/src/main/java/org/apache/calcite/util/Util.java
+++ b/core/src/main/java/org/apache/calcite/util/Util.java
@@ -2013,20 +2013,7 @@ public class Util {
    */
   public static <E> List<E> quotientList(
       final List<E> list, final int n, final int k) {
-    if (n <= 0 || k < 0 || k >= n) {
-      throw new IllegalArgumentException(
-          "n must be positive; k must be between 0 and n - 1");
-    }
-    final int size = (list.size() + n - k - 1) / n;
-    return new AbstractList<E>() {
-      @Override public E get(int index) {
-        return list.get(index * n + k);
-      }
-
-      @Override public int size() {
-        return size;
-      }
-    };
+    return new QuotientList<>(list, n, k);
   }
 
   /** Given a list with N elements
@@ -2035,7 +2022,6 @@ public class Util {
    * [ (e<sub>0</sub>, e<sub>1</sub>),
    * (e<sub>2</sub>, e<sub>3</sub>), ... ]. */
   public static <E> List<Pair<E, E>> pairs(final List<E> list) {
-    //noinspection unchecked
     return Pair.zip(quotientList(list, 2, 0),
         quotientList(list, 2, 1));
   }
@@ -2221,6 +2207,49 @@ public class Util {
     return -1;
   }
 
+  /**
+   * Returns whether the elements of {@code list} are definitely distinct
+   * and not null, working quickly and sometimes giving false negatives for
+   * large lists.
+   *
+   * <p>A return of true means that the list is distinct (true positive);
+   * a return of false means either that list is not distinct (true negative)
+   * or that the list is large and distinct (false negative).
+   * (If the list is large, a hash map would be required to do an accurate
+   * job, and this method does its best quickly.)
+   */
+  public static <E> boolean isDefinitelyDistinctAndNonNull(
+      List<? extends @Nullable E> list) {
+    final int size = list.size();
+    // An empty list is distinct.
+    if (size == 0) {
+      return true;
+    }
+    // Make sure that element zero is not null.
+    if (list.get(0) == null) {
+      return false;
+    }
+    if (size < QUICK_DISTINCT) {
+      // For smaller lists, avoid the overhead of creating a set. Threshold
+      // determined empirically using UtilTest.testIsDistinctBenchmark.
+      for (int i = 1; i < size; i++) {
+        final E e = list.get(i);
+        if (e == null) {
+          return false;
+        }
+        for (int j = i - 1; j >= 0; j--) {
+          final E e1 = list.get(j);
+          if (e.equals(e1)) {
+            return false;
+          }
+        }
+      }
+      return true;
+    }
+    // Too expensive to check.
+    return false;
+  }
+
   /** Converts a list into a list with unique elements.
    *
    * <p>The order is preserved; the second and subsequent occurrences are
@@ -2935,4 +2964,36 @@ public class Util {
       delegate.remove();
     }
   }
+
+  /** Implements {@link Util#quotientList(List, int, int)};
+   * an anonymous inner class would not be able to implement
+   * {@link RandomAccess}, which is essential for how this class is used.
+   *
+   * @param <E> Element type */
+  private static class QuotientList<E>
+      extends AbstractList<E> implements RandomAccess {
+    private final List<E> list;
+    private final int n;
+    private final int k;
+    private final int size;
+
+    QuotientList(List<E> list, int n, int k) {
+      if (k < 0 || n <= 0 || k >= n) {
+        throw new IllegalArgumentException(
+            "n must be positive; k must be between 0 and n - 1");
+      }
+      this.list = list;
+      this.n = n;
+      this.k = k;
+      this.size = (list.size() + n - k - 1) / n;
+    }
+
+    @Override public int size() {
+      return size;
+    }
+
+    @Override public E get(int index) {
+      return list.get(index * n + k);
+    }
+  }
 }
diff --git a/core/src/test/java/org/apache/calcite/util/PairListTest.java b/core/src/test/java/org/apache/calcite/util/PairListTest.java
index 3029475a89..a6319575ec 100644
--- a/core/src/test/java/org/apache/calcite/util/PairListTest.java
+++ b/core/src/test/java/org/apache/calcite/util/PairListTest.java
@@ -23,104 +23,290 @@ import com.google.common.collect.ImmutableMap;
 import org.junit.jupiter.api.Test;
 
 import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.RandomAccess;
+import java.util.function.BiPredicate;
 
+import static org.hamcrest.CoreMatchers.instanceOf;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.empty;
 import static org.hamcrest.Matchers.hasSize;
+import static org.hamcrest.Matchers.hasToString;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
 /** Unit test for {@code PairList}. */
 class PairListTest {
-  /** Basic test for {@link PairList}. */
-  @Test void testPairList() {
-    final PairList<Integer, String> pairList = PairList.of();
-    final List<Pair<Integer, String>> list = new ArrayList<>();
+  /** Equivalent to {@link Pair#left} but without calling
+   * {@link PairList#leftList()}. */
+  private static <T, U> List<T> left(
+      final List<? extends Map.Entry<? extends T, ? extends U>> pairs) {
+    return Util.transform(pairs, Map.Entry::getKey);
+  }
 
-    final Runnable validator = () -> {
-      assertThat(pairList.isEmpty(), is(list.isEmpty()));
-      assertThat(pairList, hasSize(list.size()));
-      assertThat(pairList.leftList(), hasSize(list.size()));
-      assertThat(pairList.rightList(), hasSize(list.size()));
-      assertThat(pairList.leftList(), is(Pair.left(list)));
-      assertThat(pairList.rightList(), is(Pair.right(list)));
+  /** Equivalent to {@link Pair#right} but without calling
+   * {@link PairList#rightList()}. */
+  private static <T, U> List<U> right(
+      final List<? extends Map.Entry<? extends T, ? extends U>> pairs) {
+    return Util.transform(pairs, Map.Entry::getValue);
+  }
 
-      final List<Map.Entry<Integer, String>> list2 = new ArrayList<>(pairList);
-      assertThat(list2, is(list));
+  /** Compares a {@link PairList} with a {@link List} that should have
+   * equivalent contents. */
+  private <T, U> void validate(PairList<T, U> pairList,
+      List<? extends Map.Entry<T, U>> list) {
+    assertThat(pairList.isEmpty(), is(list.isEmpty()));
+    assertThat(pairList, hasSize(list.size()));
+    assertThat(pairList.leftList(), hasSize(list.size()));
+    assertThat(pairList.rightList(), hasSize(list.size()));
+    assertThat(pairList.leftList(), is(left(list)));
+    assertThat(pairList.leftList(), instanceOf(RandomAccess.class));
+    assertThat(pairList.rightList(), is(right(list)));
+    assertThat(pairList.rightList(), instanceOf(RandomAccess.class));
 
-      // Check PairList.forEach(Consumer)
-      list2.clear();
-      //noinspection UseBulkOperation
-      pairList.forEach(p -> list2.add(p));
-      assertThat(list2, is(list));
+    final List<Map.Entry<T, U>> list2 = new ArrayList<>(pairList);
+    assertThat(list2, is(list));
 
-      // Check PairList.forEach(BiConsumer)
-      list2.clear();
-      pairList.forEach((k, v) -> list2.add(Pair.of(k, v)));
-      assertThat(list2, is(list));
+    // Check PairList.forEach(Consumer)
+    list2.clear();
+    //noinspection UseBulkOperation
+    pairList.forEach(p -> list2.add(p));
+    assertThat(list2, is(list));
 
-      // Check PairList.forEachIndexed
-      list2.clear();
-      pairList.forEachIndexed((i, k, v) -> {
-        assertThat(i, is(list2.size()));
-        list2.add(Pair.of(k, v));
-      });
-      assertThat(list2, is(list));
+    // Check PairList.forEach(BiConsumer)
+    list2.clear();
+    pairList.forEach((t, u) -> list2.add(Pair.of(t, u)));
+    assertThat(list2, is(list));
+
+    // Check PairList.forEachIndexed
+    list2.clear();
+    pairList.forEachIndexed((i, t, u) -> {
+      assertThat(i, is(list2.size()));
+      list2.add(Pair.of(t, u));
+    });
+    assertThat(list2, is(list));
 
-      final PairList<Integer, String> immutablePairList = pairList.immutable();
+    // Check PairList.immutable()
+    // Skip if there are no null keys or values
+    if (list.stream().noneMatch(e ->
+        e.getKey() == null || e.getValue() == null)) {
+      final PairList<T, U> immutablePairList = pairList.immutable();
       assertThat(immutablePairList, hasSize(list.size()));
       assertThat(immutablePairList, is(list));
-      assertThrows(UnsupportedOperationException.class, () ->
-          immutablePairList.add(0, ""));
+
       list2.clear();
       immutablePairList.forEach((k, v) -> list2.add(Pair.of(k, v)));
       assertThat(list2, is(list));
-    };
+    } else {
+      // PairList.immutable should throw if there are null keys or values
+      assertThrows(NullPointerException.class,
+          pairList::immutable);
+    }
+  }
 
-    validator.run();
+  /** Basic test for {@link PairList}. */
+  @Test void testPairList() {
+    final PairList<Integer, String> pairList = PairList.of();
+    final List<Map.Entry<Integer, String>> list = new ArrayList<>();
 
+    validate(pairList, list);
+
+    // add(T, U)
     pairList.add(1, "a");
     list.add(Pair.of(1, "a"));
-    validator.run();
+    validate(pairList, list);
 
+    // add(Pair<T, U>)
     pairList.add(Pair.of(2, "b"));
     list.add(Pair.of(2, "b"));
-    validator.run();
+    validate(pairList, list);
+
+    // add(T, U)
+    pairList.add(2, "bb");
+    list.add(Pair.of(2, "bb"));
+    validate(pairList, list);
 
+    // add(int, Pair<T, U>)
     pairList.add(0, Pair.of(3, "c"));
     list.add(0, Pair.of(3, "c"));
-    validator.run();
+    validate(pairList, list);
+
+    // add(int, T, U)
+    pairList.add(0, 4, "d");
+    list.add(0, Pair.of(4, "d"));
+    validate(pairList, list);
 
+    // remove(int)
     Map.Entry<Integer, String> x = pairList.remove(1);
-    Pair<Integer, String> y = list.remove(1);
+    Map.Entry<Integer, String> y = list.remove(1);
     assertThat(x, is(y));
-    validator.run();
+    validate(pairList, list);
 
+    // clear()
     pairList.clear();
     list.clear();
-    validator.run();
+    validate(pairList, list);
+
+    // clear() again
+    pairList.clear();
+    list.clear();
+    validate(pairList, list);
+
+    // add(T, U) having called clear
+    pairList.add(-1, "c");
+    list.add(Pair.of(-1, "c"));
+    validate(pairList, list);
+
+    // addAll(PairList)
+    final PairList<Integer, String> pairList8 = PairList.copyOf(8, "x", 7, "y");
+    pairList.addAll(pairList8);
+    list.addAll(pairList8);
+    validate(pairList, list);
+
+    // addAll(int, PairList)
+    pairList.addAll(3, pairList8);
+    list.addAll(3, pairList8);
+    validate(pairList, list);
+
+    PairList<Integer, String> immutablePairList = pairList.immutable();
+    assertThrows(UnsupportedOperationException.class, () ->
+        immutablePairList.add(0, ""));
+    validate(immutablePairList, list);
+
+    // set(int, Pair<T, U>)
+    pairList.set(2, 0, "p");
+    list.set(2, Pair.of(0, "p"));
+    validate(pairList, list);
+
+    // set(int, T, U)
+    pairList.set(1, Pair.of(88, "q"));
+    list.set(1, Pair.of(88, "q"));
+    validate(pairList, list);
   }
 
   /** Tests {@link PairList#of(Map)} and {@link PairList#toImmutableMap()}. */
   @Test void testPairListOfMap() {
     final ImmutableMap<String, Integer> map = ImmutableMap.of("a", 1, "b", 2);
-    final PairList<String, Integer> list = PairList.of(map);
-    assertThat(list, hasSize(2));
-    assertThat(list.toString(), is("[<a, 1>, <b, 2>]"));
+    final PairList<String, Integer> pairList = PairList.of(map);
+    assertThat(pairList, hasSize(2));
+    assertThat(pairList.toString(), is("[<a, 1>, <b, 2>]"));
+
+    final List<Map.Entry<String, Integer>> list = new ArrayList<>(map.entrySet());
+    validate(pairList, list);
 
-    final ImmutableMap<String, Integer> map2 = list.toImmutableMap();
+    final ImmutableMap<String, Integer> map2 = pairList.toImmutableMap();
     assertThat(map2, is(map));
 
     // After calling toImmutableMap, you can modify the list and call
     // toImmutableMap again.
-    list.add("c", 3);
-    assertThat(list.toString(), is("[<a, 1>, <b, 2>, <c, 3>]"));
-    final ImmutableMap<String, Integer> map3 = list.toImmutableMap();
+    pairList.add("c", 3);
+    list.add(Pair.of("c", 3));
+    validate(pairList, list);
+    assertThat(pairList.toString(), is("[<a, 1>, <b, 2>, <c, 3>]"));
+    final ImmutableMap<String, Integer> map3 = pairList.toImmutableMap();
     assertThat(map3.toString(), is("{a=1, b=2, c=3}"));
 
     final Map<String, Integer> emptyMap = ImmutableMap.of();
-    final PairList<String, Integer> emptyList = PairList.of(emptyMap);
-    assertThat(emptyList.isEmpty(), is(true));
+    final PairList<String, Integer> emptyPairList = PairList.of(emptyMap);
+    assertThat(emptyPairList.isEmpty(), is(true));
+    validate(emptyPairList, Collections.emptyList());
+  }
+
+  /** Tests {@link PairList#withCapacity(int)}. */
+  @Test void testPairListWithCapacity() {
+    final PairList<String, Integer> list = PairList.withCapacity(100);
+    assertThat(list, hasSize(0));
+    assertThat(list, empty());
+    assertThat(list, hasToString("[]"));
+
+    list.add("a", 1);
+    list.add("b", 2);
+    assertThat(list, hasSize(2));
+    assertThat(list, hasToString("[<a, 1>, <b, 2>]"));
+
+    final Map.Entry<String, Integer> entry = list.remove(0);
+    assertThat(entry.getKey(), is("a"));
+    assertThat(entry.getValue(), is(1));
+    assertThat(list, hasToString("[<b, 2>]"));
+  }
+
+  @Test void testPairListOf() {
+    final PairList<String, Integer> list0 = PairList.of();
+    assertThat(list0, hasSize(0));
+    assertThat(list0, empty());
+    assertThat(list0, hasToString("[]"));
+
+    final PairList<String, Integer> list1 = PairList.of("a", 1);
+    assertThat(list1, hasSize(1));
+    assertThat(list1, hasToString("[<a, 1>]"));
+
+    final PairList<String, Integer> list3 =
+        PairList.copyOf("a", 1, "b", null, "c", 3);
+    assertThat(list3, hasSize(3));
+    assertThat(list3, hasToString("[<a, 1>, <b, null>, <c, 3>]"));
+
+    assertThrows(IllegalArgumentException.class,
+        () -> PairList.copyOf("a", 1, "b", 2, "c"),
+        "odd number of arguments");
+  }
+
+  @Test void testTransform() {
+    final PairList<String, Integer> list3 =
+        PairList.copyOf("a", 1, null, 5, "c", 3);
+    assertThat(list3.transform((s, i) -> s + i),
+        is(Arrays.asList("a1", "null5", "c3")));
+
+    final PairList<String, Integer> list0 = PairList.of();
+    assertThat(list0.transform((s, i) -> s + i), empty());
+
+    final BiPredicate<String, Integer> gt2 = (s, i) -> i > 2;
+    assertThat(list3.anyMatch(gt2), is(true));
+    assertThat(list3.allMatch(gt2), is(false));
+    assertThat(list3.noMatch(gt2), is(false));
+
+    final BiPredicate<String, Integer> negative = (s, i) -> i < 0;
+    assertThat(list3.anyMatch(negative), is(false));
+    assertThat(list3.allMatch(negative), is(false));
+    assertThat(list3.noMatch(negative), is(true));
+
+    final BiPredicate<String, Integer> positive = (s, i) -> i > 0;
+    assertThat(list3.anyMatch(positive), is(true));
+    assertThat(list3.allMatch(positive), is(true));
+    assertThat(list3.noMatch(positive), is(false));
+
+    final BiPredicate<String, Integer> isNull = (s, i) -> s == null;
+    assertThat(list3.anyMatch(isNull), is(true));
+    assertThat(list3.allMatch(isNull), is(false));
+    assertThat(list3.noMatch(isNull), is(false));
+
+    // All predicates behave the same on the empty list
+    Arrays.asList(gt2, negative, positive, isNull).forEach(p -> {
+      assertThat(list0.anyMatch(p), is(false));
+      assertThat(list0.allMatch(p), is(true)); // trivially
+      assertThat(list0.noMatch(p), is(true));
+    });
+  }
+
+  @Test void testBuilder() {
+    final PairList.Builder<String, Integer> b = PairList.builder();
+    final List<Pair<String, Integer>> list = new ArrayList<>();
+
+    final PairList<String, Integer> list0 = b.build();
+    validate(list0, list);
+
+    b.add("a", 1);
+    list.add(Pair.of("a", 1));
+    final PairList<String, Integer> list1 = b.build();
+    validate(list1, list);
+
+    b.add("b", 2);
+    b.add("c", null);
+    list.add(Pair.of("b", 2));
+    list.add(Pair.of("c", null));
+    final PairList<String, Integer> list3 = b.build();
+    validate(list3, list);
   }
 }
diff --git a/core/src/test/java/org/apache/calcite/util/UtilTest.java b/core/src/test/java/org/apache/calcite/util/UtilTest.java
index a08cf1a8e8..d5cdff4c34 100644
--- a/core/src/test/java/org/apache/calcite/util/UtilTest.java
+++ b/core/src/test/java/org/apache/calcite/util/UtilTest.java
@@ -22,6 +22,7 @@ import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
 import org.apache.calcite.linq4j.Ord;
+import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.linq4j.function.Parameter;
 import org.apache.calcite.runtime.ConsList;
 import org.apache.calcite.runtime.FlatLists;
@@ -99,6 +100,7 @@ import java.util.function.BiConsumer;
 import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Function;
+import java.util.function.IntFunction;
 import java.util.function.ObjIntConsumer;
 import java.util.function.Predicate;
 import java.util.function.UnaryOperator;
@@ -1875,6 +1877,38 @@ class UtilTest {
     assertThat(Util.distinctList(a1m), is(a));
   }
 
+  @Test void testIsDefinitelyDistinctAndNonNull() {
+    final BiConsumer<List<String>, Boolean> f = (list, b) -> {
+      assertThat(Util.isDefinitelyDistinctAndNonNull(list), is(b));
+    };
+
+    f.accept(ImmutableList.of(), true);
+    f.accept(Collections.singletonList(null), false);
+    f.accept(list("a", "b", "a"), false);
+    f.accept(list("a", "b", "c"), true);
+    f.accept(list(null, "b", "c"), false);
+    f.accept(list(null, "b", null), false);
+    f.accept(list(null, null), false);
+    final IntFunction<String> stringFn =
+        i -> String.valueOf((char) ('a' + i));
+    // {"a", "b", "c", "d", "e", "f"} is distinct
+    final List<String> alpha6 = Functions.generate(6, stringFn);
+    f.accept(alpha6, true);
+    // {"a", ... "n"} is distinct
+    final List<String> alpha14 =
+        new ArrayList<>(Functions.generate(14, stringFn));
+    f.accept(alpha14, true);
+    alpha14.set(10, "a");
+    f.accept(alpha14, false);
+    alpha14.set(10, null);
+    f.accept(alpha14, false);
+    alpha14.set(10, "z");
+    f.accept(alpha14, true);
+    // {"a", ... "t"} is distinct but has length above the threshold,
+    // so gives a false negative
+    f.accept(Functions.generate(20, stringFn), false);
+  }
+
   /** Unit test for {@link Utilities#hashCode(double)}. */
   @Test void testHash() {
     checkHash(0d);
diff --git a/innodb/src/main/java/org/apache/calcite/adapter/innodb/InnodbToEnumerableConverter.java b/innodb/src/main/java/org/apache/calcite/adapter/innodb/InnodbToEnumerableConverter.java
index 2c5c1874cc..5a32d81584 100644
--- a/innodb/src/main/java/org/apache/calcite/adapter/innodb/InnodbToEnumerableConverter.java
+++ b/innodb/src/main/java/org/apache/calcite/adapter/innodb/InnodbToEnumerableConverter.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterImpl;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
@@ -46,10 +47,11 @@ import com.alibaba.innodb.java.reader.comparator.ComparisonOperator;
 import org.checkerframework.checker.nullness.qual.Nullable;
 
 import java.util.AbstractList;
-import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
 
 /**
  * Relational expression representing a scan of a table
@@ -102,12 +104,8 @@ public class InnodbToEnumerableConverter extends ConverterImpl
                       }
                     }),
                 Pair.class));
-    List<Map.Entry<String, String>> selectList = new ArrayList<>();
-    for (Map.Entry<String, String> entry
-        : Pair.zip(innodbImplementor.selectFields.keySet(),
-        innodbImplementor.selectFields.values())) {
-      selectList.add(entry);
-    }
+    PairList<String, String> selectList =
+        PairList.of(innodbImplementor.selectFields);
     final Expression selectFields =
         list.append("selectFields", constantArrayList(selectList, Pair.class));
     final Expression table =
@@ -160,9 +158,23 @@ public class InnodbToEnumerableConverter extends ConverterImpl
    * E.g. {@code constantArrayList("x", "y")} returns
    * "Arrays.asList('x', 'y')".
    */
+  @SuppressWarnings({"rawtypes", "unchecked"})
   private static <T> Expression constantArrayList(List<T> values, Class clazz) {
-    return Expressions.call(
-        BuiltInMethod.ARRAYS_AS_LIST.method,
+    if (values instanceof PairList
+        && !values.isEmpty()
+        && Map.Entry.class.isAssignableFrom(clazz)) {
+      // For PairList, we cannot generate Arrays.asList because Map.Entry does
+      // not an obvious implementation with default constructor. Instead,
+      // generate
+      //   PairList.of("k0", "v0", "k1", "v1");
+      final List<Object> keyValues =
+          ((PairList<Object, Object>) values).stream()
+              .flatMap(p -> Stream.of(p.getKey(), p.getValue()))
+              .collect(Collectors.toList());
+      return Expressions.call(null, BuiltInMethod.PAIR_LIST_COPY_OF.method,
+          constantList(keyValues));
+    }
+    return Expressions.call(BuiltInMethod.ARRAYS_AS_LIST.method,
         Expressions.newArrayInit(clazz, constantList(values)));
   }
 
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Nullness.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Nullness.java
index 4ae66f400f..c158e39a2b 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Nullness.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Nullness.java
@@ -22,6 +22,8 @@ import org.checkerframework.checker.nullness.qual.NonNull;
 import org.checkerframework.checker.nullness.qual.Nullable;
 import org.checkerframework.dataflow.qual.Pure;
 
+import java.util.List;
+
 /**
  * The methods in this class allow to cast nullable reference to a non-nullable one.
  * This is an internal class, and it is not meant to be used as a public API.
@@ -82,13 +84,28 @@ public class Nullness {
    * @param ts      Array
    * @return the argument, cast so that elements are @NonNull
    */
+  @SuppressWarnings({"unchecked", "ConstantConditions"})
   @Pure
   public static <T> @NonNull T[] castNonNullArray(
       @Nullable T[] ts) {
-    //noinspection ConstantConditions,unchecked
     return (@NonNull T []) (Object) ts;
   }
 
+  /**
+   * Allows you to treat a list of nullable values as an list of non-nullable
+   * values.
+   *
+   * @param <T>     Type of the list elements
+   * @param ts      List
+   * @return the argument, cast so that elements are @NonNull
+   */
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  @Pure
+  public static <T> List<@NonNull T> castNonNullList(
+      List<? extends @Nullable T> ts) {
+    return (List) (Object) ts;
+  }
+
   /**
    * Allows you to treat an uninitialized or under-initialization object as
    * initialized with no assertions.
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
index 2f40afa168..2a7cb3a9c1 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoRel.java
@@ -20,10 +20,7 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexBuilder;
-import org.apache.calcite.util.Pair;
-
-import java.util.ArrayList;
-import java.util.List;
+import org.apache.calcite.runtime.PairList;
 
 /**
  * Relational expression that uses Mongo calling convention.
@@ -37,7 +34,7 @@ public interface MongoRel extends RelNode {
   /** Callback for the implementation process that converts a tree of
    * {@link MongoRel} nodes into a MongoDB query. */
   class Implementor {
-    final List<Pair<String, String>> list = new ArrayList<>();
+    final PairList<String, String> list = PairList.of();
     final RexBuilder rexBuilder;
     RelOptTable table;
     MongoTable mongoTable;
@@ -47,7 +44,7 @@ public interface MongoRel extends RelNode {
     }
 
     public void add(String findOp, String aggOp) {
-      list.add(Pair.of(findOp, aggOp));
+      list.add(findOp, aggOp);
     }
 
     public void visitChild(int ordinal, RelNode input) {
diff --git a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
index 319384d096..116bd6e5dd 100644
--- a/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
+++ b/mongodb/src/main/java/org/apache/calcite/adapter/mongodb/MongoToEnumerableConverter.java
@@ -106,7 +106,7 @@ public class MongoToEnumerableConverter
         list.append("table",
             mongoImplementor.table.getExpression(
                 MongoTable.MongoQueryable.class));
-    List<String> opList = Pair.right(mongoImplementor.list);
+    List<String> opList = mongoImplementor.list.rightList();
     final Expression ops =
         list.append("ops",
             constantArrayList(opList, String.class));
diff --git a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
index 5422472c99..44400635eb 100644
--- a/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
+++ b/splunk/src/main/java/org/apache/calcite/adapter/splunk/SplunkPushDownRule.java
@@ -31,6 +31,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexSlot;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.sql.SqlBinaryOperator;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -38,7 +39,6 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.NlsString;
-import org.apache.calcite.util.Pair;
 
 import com.google.common.collect.ImmutableSet;
 
@@ -247,7 +247,7 @@ public class SplunkPushDownRule
     }
 
     // field renaming: to -> from
-    List<Pair<String, String>> renames = new ArrayList<>();
+    final PairList<String, String> renames = PairList.of();
 
     // handle top projection (ie reordering and renaming)
     List<RelDataTypeField> newFields = bottomFields;
@@ -260,10 +260,8 @@ public class SplunkPushDownRule
         RelDataTypeField field = bottomFields.get(rif.getIndex());
         if (!bottomFields.get(rif.getIndex()).getName()
             .equals(topFields.get(i).getName())) {
-          renames.add(
-              Pair.of(
-                  bottomFields.get(rif.getIndex()).getName(),
-                  topFields.get(i).getName()));
+          renames.add(bottomFields.get(rif.getIndex()).getName(),
+              topFields.get(i).getName());
           field = topFields.get(i);
         }
         newFields.add(field);
@@ -272,10 +270,9 @@ public class SplunkPushDownRule
 
     if (!renames.isEmpty()) {
       updateSearchStr.append("| rename ");
-      for (Pair<String, String> p : renames) {
-        updateSearchStr.append(p.left).append(" AS ")
-            .append(p.right).append(" ");
-      }
+      renames.forEach((left, right) ->
+          updateSearchStr.append(left).append(" AS ")
+              .append(right).append(" "));
     }
 
     RelDataType resultType =
diff --git a/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java b/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
index 3904f43870..02d32c9eca 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/test/AbstractSqlTester.java
@@ -22,6 +22,7 @@ import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.runtime.Utilities;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlLiteral;
@@ -407,7 +408,7 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
       return -Utilities.compare(pos0.getColumnNum(), pos1.getColumnNum());
     });
     String sql2 = sql;
-    final List<Pair<String, String>> values = new ArrayList<>();
+    final PairList<String, String> values = PairList.of();
     int p = 0;
     for (SqlNode literal : nodes) {
       final SqlParserPos pos = literal.getParserPosition();
@@ -420,20 +421,20 @@ public abstract class AbstractSqlTester implements SqlTester, AutoCloseable {
               pos.getEndLineNum(),
               pos.getEndColumnNum()) + 1;
       String param = "p" + p++;
-      values.add(Pair.of(sql2.substring(start, end), param));
+      values.add(sql2.substring(start, end), param);
       sql2 = sql2.substring(0, start)
           + param
           + sql2.substring(end);
     }
     if (values.isEmpty()) {
-      values.add(Pair.of("1", "p0"));
+      values.add("1", "p0");
     }
     return "select "
         + sql2.substring("values (".length(), sql2.length() - 1)
         + " from (values ("
-        + Util.commaList(Pair.left(values))
+        + Util.commaList(values.leftList())
         + ")) as t("
-        + Util.commaList(Pair.right(values))
+        + Util.commaList(values.rightList())
         + ")";
   }
 
diff --git a/testkit/src/main/java/org/apache/calcite/test/DiffRepository.java b/testkit/src/main/java/org/apache/calcite/test/DiffRepository.java
index 42af61dac3..f1052ea804 100644
--- a/testkit/src/main/java/org/apache/calcite/test/DiffRepository.java
+++ b/testkit/src/main/java/org/apache/calcite/test/DiffRepository.java
@@ -189,7 +189,7 @@ public class DiffRepository {
 
   //~ Instance fields --------------------------------------------------------
 
-  private final DiffRepository baseRepository;
+  private final @Nullable DiffRepository baseRepository;
   private final int indent;
   private final ImmutableSortedSet<String> outOfOrderTests;
   private Document doc;
@@ -210,7 +210,7 @@ public class DiffRepository {
    * @param indent    Indentation of XML file
    */
   private DiffRepository(URL refFile, File logFile,
-      DiffRepository baseRepository, Filter filter, int indent) {
+      @Nullable DiffRepository baseRepository, Filter filter, int indent) {
     this.baseRepository = baseRepository;
     this.filter = filter;
     this.indent = indent;
@@ -404,7 +404,7 @@ public class DiffRepository {
    *                      a base repository, it has overrides="true"
    * @return TestCase element, or null if not found
    */
-  private synchronized Element getTestCaseElement(
+  private synchronized @Nullable Element getTestCaseElement(
       final String testCaseName,
       boolean checkOverride,
       List<Pair<String, Element>> elements) {
diff --git a/testkit/src/main/java/org/apache/calcite/test/catalog/CompoundNameColumnResolver.java b/testkit/src/main/java/org/apache/calcite/test/catalog/CompoundNameColumnResolver.java
index 331b190f63..1a19f084be 100644
--- a/testkit/src/main/java/org/apache/calcite/test/catalog/CompoundNameColumnResolver.java
+++ b/testkit/src/main/java/org/apache/calcite/test/catalog/CompoundNameColumnResolver.java
@@ -22,11 +22,13 @@ import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rel.type.RelDataTypeFieldImpl;
 import org.apache.calcite.rel.type.StructKind;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.util.Pair;
 
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Comparator;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -94,26 +96,19 @@ final class CompoundNameColumnResolver implements MockCatalogReader.ColumnResolv
       }
       index = entry.getValue().get(columnName);
       if (index != null) {
-        ret.add(
-            new Pair<RelDataTypeField, List<String>>(
-                rowType.getFieldList().get(index), remainder));
+        ret.add(new Pair<>(rowType.getFieldList().get(index), remainder));
       }
     }
 
     if (ret.isEmpty() && names.size() == 1) {
       Map<String, Integer> subMap = groupMap.get(columnName);
       if (subMap != null) {
-        List<Map.Entry<String, Integer>> entries =
-            new ArrayList<>(subMap.entrySet());
-        entries.sort((o1, o2) -> o1.getValue() - o2.getValue());
+        PairList<String, Integer> entries = PairList.of(subMap);
+        entries.sort(Comparator.comparingInt(Map.Entry::getValue));
         ret.add(
-            new Pair<RelDataTypeField, List<String>>(
-                new RelDataTypeFieldImpl(
-                    columnName, -1,
-                    createStructType(
-                        rowType,
-                        typeFactory,
-                        entries)),
+            new Pair<>(
+                new RelDataTypeFieldImpl(columnName, -1,
+                    createStructType(rowType, typeFactory, entries)),
                 remainder));
       }
     }
diff --git a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
index fc72970326..19b68fa775 100644
--- a/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
+++ b/testkit/src/main/java/org/apache/calcite/test/catalog/MockCatalogReader.java
@@ -54,6 +54,7 @@ import org.apache.calcite.rel.type.StructKind;
 import org.apache.calcite.rex.RexBuilder;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.schema.CustomColumnResolvingTable;
 import org.apache.calcite.schema.ExtensibleTable;
 import org.apache.calcite.schema.Path;
@@ -948,20 +949,12 @@ public abstract class MockCatalogReader extends CalciteCatalogReader {
           LogicalFilter.create(rel, getConstraint(rexBuilder, rel.getRowType()));
       final List<RelDataTypeField> fieldList =
           rel.getRowType().getFieldList();
-      final List<Pair<RexNode, String>> projects =
-          new AbstractList<Pair<RexNode, String>>() {
-            @Override public Pair<RexNode, String> get(int index) {
-              return RexInputRef.of2(mapping.get(index), fieldList);
-            }
-
-            @Override public int size() {
-              return mapping.size();
-            }
-          };
+      final PairList<RexNode, String> projects = PairList.of();
+      mapping.forEachInt(i -> RexInputRef.add2(projects, i, fieldList));
       return LogicalProject.create(rel,
           ImmutableList.of(),
-          Pair.left(projects),
-          Pair.right(projects),
+          projects.leftList(),
+          projects.rightList(),
           ImmutableSet.of());
     }
 
diff --git a/testkit/src/main/java/org/apache/calcite/util/Smalls.java b/testkit/src/main/java/org/apache/calcite/util/Smalls.java
index 984de925a6..791b19d313 100644
--- a/testkit/src/main/java/org/apache/calcite/util/Smalls.java
+++ b/testkit/src/main/java/org/apache/calcite/util/Smalls.java
@@ -40,6 +40,7 @@ import org.apache.calcite.rel.externalize.RelJsonReader;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.runtime.PairList;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.schema.FunctionContext;
 import org.apache.calcite.schema.FunctionParameter;
@@ -71,7 +72,6 @@ import java.sql.Date;
 import java.sql.Time;
 import java.sql.Timestamp;
 import java.util.AbstractList;
-import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
@@ -1381,12 +1381,12 @@ public class Smalls {
 
     @Override public RelDataType getRowType(RelDataTypeFactory typeFactory) {
       int columnCount = columnNames.length;
-      final List<Pair<String, RelDataType>> columnDesc =
-          new ArrayList<>(columnCount);
+      final PairList<String, RelDataType> columnDesc =
+          PairList.withCapacity(columnCount);
       for (int i = 0; i < columnCount; i++) {
         final RelDataType colType = typeFactory
             .createJavaType(columnTypes[i]);
-        columnDesc.add(Pair.of(columnNames[i], colType));
+        columnDesc.add(columnNames[i], colType);
       }
       return typeFactory.createStructType(columnDesc);
     }