You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by mm...@apache.org on 2018/07/20 17:41:31 UTC

[18/53] [abbrv] calcite git commit: [CALCITE-2259] Allow Java 8 syntax

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
index 81d84ad..aeeca3d 100644
--- a/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/RelOptTableImpl.java
@@ -60,15 +60,14 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
-import com.google.common.base.Functions;
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.AbstractList;
 import java.util.Collection;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
+import java.util.function.Function;
 
 /**
  * Implementation of {@link org.apache.calcite.plan.RelOptTable}.
@@ -97,7 +96,7 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
       Function<Class, Expression> expressionFunction,
       Double rowCount) {
     this.schema = schema;
-    this.rowType = Preconditions.checkNotNull(rowType);
+    this.rowType = Objects.requireNonNull(rowType);
     this.names = ImmutableList.copyOf(names);
     this.table = table; // may be null
     this.expressionFunction = expressionFunction; // may be null
@@ -109,29 +108,23 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
       RelDataType rowType,
       List<String> names,
       Expression expression) {
-    //noinspection unchecked
-    final Function<Class, Expression> expressionFunction =
-        (Function) Functions.constant(expression);
     return new RelOptTableImpl(schema, rowType, names, null,
-        expressionFunction, null);
+        c -> expression, null);
   }
 
   public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType,
       Table table, Path path) {
     final SchemaPlus schemaPlus = MySchemaPlus.create(path);
-    Function<Class, Expression> expressionFunction =
-        getClassExpressionFunction(schemaPlus, Util.last(path).left, table);
     return new RelOptTableImpl(schema, rowType, Pair.left(path), table,
-        expressionFunction, table.getStatistic().getRowCount());
+        getClassExpressionFunction(schemaPlus, Util.last(path).left, table),
+        table.getStatistic().getRowCount());
   }
 
   public static RelOptTableImpl create(RelOptSchema schema, RelDataType rowType,
       final CalciteSchema.TableEntry tableEntry, Double rowCount) {
     final Table table = tableEntry.getTable();
-    Function<Class, Expression> expressionFunction =
-        getClassExpressionFunction(tableEntry, table);
     return new RelOptTableImpl(schema, rowType, tableEntry.path(),
-        table, expressionFunction, rowCount);
+        table, getClassExpressionFunction(tableEntry, table), rowCount);
   }
 
   /**
@@ -152,28 +145,18 @@ public class RelOptTableImpl extends Prepare.AbstractPreparingTable {
       final SchemaPlus schema, final String tableName, final Table table) {
     if (table instanceof QueryableTable) {
       final QueryableTable queryableTable = (QueryableTable) table;
-      return new Function<Class, Expression>() {
-        public Expression apply(Class clazz) {
-          return queryableTable.getExpression(schema, tableName, clazz);
-        }
-      };
+      return clazz -> queryableTable.getExpression(schema, tableName, clazz);
     } else if (table instanceof ScannableTable
         || table instanceof FilterableTable
         || table instanceof ProjectableFilterableTable) {
-      return new Function<Class, Expression>() {
-        public Expression apply(Class clazz) {
-          return Schemas.tableExpression(schema, Object[].class, tableName,
-              table.getClass());
-        }
-      };
+      return clazz -> Schemas.tableExpression(schema, Object[].class, tableName,
+          table.getClass());
     } else if (table instanceof StreamableTable) {
       return getClassExpressionFunction(schema, tableName,
           ((StreamableTable) table).stream());
     } else {
-      return new Function<Class, Expression>() {
-        public Expression apply(Class input) {
-          throw new UnsupportedOperationException();
-        }
+      return input -> {
+        throw new UnsupportedOperationException();
       };
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/profile/ProfilerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/profile/ProfilerImpl.java b/core/src/main/java/org/apache/calcite/profile/ProfilerImpl.java
index 139f13d..2dd8af9 100644
--- a/core/src/main/java/org/apache/calcite/profile/ProfilerImpl.java
+++ b/core/src/main/java/org/apache/calcite/profile/ProfilerImpl.java
@@ -22,16 +22,12 @@ import org.apache.calcite.materialize.Lattice;
 import org.apache.calcite.prepare.CalcitePrepareImpl;
 import org.apache.calcite.rel.metadata.NullSentinel;
 import org.apache.calcite.runtime.FlatLists;
-import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.PartiallyOrderedSet;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
@@ -46,7 +42,6 @@ import java.util.Arrays;
 import java.util.BitSet;
 import java.util.Collection;
 import java.util.Collections;
-import java.util.Comparator;
 import java.util.Deque;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -57,6 +52,7 @@ import java.util.Queue;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
+import java.util.function.Predicate;
 
 import static org.apache.calcite.profile.ProfilerImpl.CompositeCollector.OF;
 
@@ -119,20 +115,17 @@ public class ProfilerImpl implements Profiler {
      * not yet been computed. We may add some of those successors to
      * {@link #spaceQueue}. */
     final Queue<Space> doneQueue =
-        new PriorityQueue<>(100,
-          new Comparator<Space>() {
-            public int compare(Space s0, Space s1) {
-              // The space with 0 columns is more interesting than
-              // any space with 1 column, and so forth.
-              // For spaces with 2 or more columns we compare "surprise":
-              // how many fewer values did it have than expected?
-              int c = Integer.compare(s0.columns.size(), s1.columns.size());
-              if (c == 0) {
-                c = Double.compare(s0.surprise(), s1.surprise());
-              }
-              return c;
-            }
-          });
+        new PriorityQueue<>(100, (s0, s1) -> {
+          // The space with 0 columns is more interesting than
+          // any space with 1 column, and so forth.
+          // For spaces with 2 or more columns we compare "surprise":
+          // how many fewer values did it have than expected?
+          int c = Integer.compare(s0.columns.size(), s1.columns.size());
+          if (c == 0) {
+            c = Double.compare(s0.surprise(), s1.surprise());
+          }
+          return c;
+        });
     final SurpriseQueue surprises;
 
     /** Combinations of columns that we will compute next pass. */
@@ -143,20 +136,11 @@ public class ProfilerImpl implements Profiler {
      * Ensures that we do not calculate the same combination more than once,
      * even though we generate a column set from multiple parents. */
     final Set<ImmutableBitSet> resultSet = new HashSet<>();
-    final PartiallyOrderedSet<Space> results = new PartiallyOrderedSet<>(
-        new PartiallyOrderedSet.Ordering<Space>() {
-          public boolean lessThan(Space e1, Space e2) {
-            return e2.columnOrdinals.contains(e1.columnOrdinals);
-          }
-        });
+    final PartiallyOrderedSet<Space> results =
+        new PartiallyOrderedSet<>((e1, e2) ->
+            e2.columnOrdinals.contains(e1.columnOrdinals));
     private final List<ImmutableBitSet> keyOrdinalLists =
         new ArrayList<>();
-    final Function<Integer, Column> get =
-        new Function<Integer, Column>() {
-          public Column apply(Integer input) {
-            return columns.get(input);
-          }
-        };
     private int rowCount;
 
     /**
@@ -262,7 +246,7 @@ public class ProfilerImpl implements Profiler {
                     || doneSpace.columnOrdinals.cardinality() == 0
                     || !containsKey(
                         doneSpace.columnOrdinals.set(column.ordinal))
-                    && predicate.apply(Pair.of(doneSpace, column))) {
+                    && predicate.test(Pair.of(doneSpace, column))) {
                   final ImmutableBitSet nextOrdinals =
                       doneSpace.columnOrdinals.set(column.ordinal);
                   if (resultSet.add(nextOrdinals)) {
@@ -448,7 +432,8 @@ public class ProfilerImpl implements Profiler {
 
 
     private ImmutableSortedSet<Column> toColumns(Iterable<Integer> ordinals) {
-      return ImmutableSortedSet.copyOf(Iterables.transform(ordinals, get));
+      return ImmutableSortedSet.copyOf(
+          Iterables.transform(ordinals, columns::get));
     }
   }
 
@@ -500,7 +485,7 @@ public class ProfilerImpl implements Profiler {
   /** Builds a {@link org.apache.calcite.profile.ProfilerImpl}. */
   public static class Builder {
     int combinationsPerPass = 100;
-    Predicate<Pair<Space, Column>> predicate = Predicates.alwaysTrue();
+    Predicate<Pair<Space, Column>> predicate = p -> true;
 
     public ProfilerImpl build() {
       return new ProfilerImpl(combinationsPerPass, 200, predicate);
@@ -513,11 +498,9 @@ public class ProfilerImpl implements Profiler {
 
     public Builder withMinimumSurprise(double v) {
       predicate =
-          new PredicateImpl<Pair<Space, Column>>() {
-            public boolean test(Pair<Space, Column> spaceColumnPair) {
-              final Space space = spaceColumnPair.left;
-              return false;
-            }
+          spaceColumnPair -> {
+            final Space space = spaceColumnPair.left;
+            return false;
           };
       return this;
     }
@@ -624,7 +607,7 @@ public class ProfilerImpl implements Profiler {
             new ArrayList<>(
                 Collections.nCopies(columnOrdinals[columnOrdinals.length - 1]
                         + 1,
-                    (Comparable) null));
+                    null));
         for (FlatLists.ComparableList value : this.values) {
           for (int i = 0; i < value.size(); i++) {
             Comparable c = (Comparable) value.get(i);
@@ -759,7 +742,7 @@ public class ProfilerImpl implements Profiler {
     int count = 0;
     final Deque<Double> deque = new ArrayDeque<>();
     final PriorityQueue<Double> priorityQueue =
-        new PriorityQueue<>(11, Ordering.<Double>natural());
+        new PriorityQueue<>(11, Ordering.natural());
 
     SurpriseQueue(int warmUpCount, int size) {
       this.warmUpCount = warmUpCount;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/profile/SimpleProfiler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/profile/SimpleProfiler.java b/core/src/main/java/org/apache/calcite/profile/SimpleProfiler.java
index c9b00d0..f1c7ee4 100644
--- a/core/src/main/java/org/apache/calcite/profile/SimpleProfiler.java
+++ b/core/src/main/java/org/apache/calcite/profile/SimpleProfiler.java
@@ -24,7 +24,6 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.PartiallyOrderedSet;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ImmutableSortedSet;
 import com.google.common.collect.Iterables;
 
@@ -45,12 +44,6 @@ import javax.annotation.Nonnull;
  * Basic implementation of {@link Profiler}.
  */
 public class SimpleProfiler implements Profiler {
-  private static final Function<List<Comparable>, Comparable> ONLY =
-      new Function<List<Comparable>, Comparable>() {
-        public Comparable apply(List<Comparable> input) {
-          return Iterables.getOnlyElement(input);
-        }
-      };
 
   public Profile profile(Iterable<List<Comparable>> rows,
       final List<Column> columns, Collection<ImmutableBitSet> initialGroups) {
@@ -97,23 +90,13 @@ public class SimpleProfiler implements Profiler {
     final List<Space> singletonSpaces;
     final List<Statistic> statistics = new ArrayList<>();
     final PartiallyOrderedSet.Ordering<Space> ordering =
-        new PartiallyOrderedSet.Ordering<Space>() {
-          public boolean lessThan(Space e1, Space e2) {
-            return e2.columnOrdinals.contains(e1.columnOrdinals);
-          }
-        };
+        (e1, e2) -> e2.columnOrdinals.contains(e1.columnOrdinals);
     final PartiallyOrderedSet<Space> results =
         new PartiallyOrderedSet<>(ordering);
     final PartiallyOrderedSet<Space> keyResults =
         new PartiallyOrderedSet<>(ordering);
     private final List<ImmutableBitSet> keyOrdinalLists =
         new ArrayList<>();
-    final Function<Integer, Column> get =
-        new Function<Integer, Column>() {
-          public Column apply(Integer input) {
-            return columns.get(input);
-          }
-        };
 
     Run(final List<Column> columns) {
       for (Ord<Column> column : Ord.zip(columns)) {
@@ -123,7 +106,7 @@ public class SimpleProfiler implements Profiler {
       }
       this.columns = columns;
       this.singletonSpaces =
-          new ArrayList<>(Collections.nCopies(columns.size(), (Space) null));
+          new ArrayList<>(Collections.nCopies(columns.size(), null));
       for (ImmutableBitSet ordinals
           : ImmutableBitSet.range(columns.size()).powerSet()) {
         final Space space = new Space(ordinals, toColumns(ordinals));
@@ -217,7 +200,7 @@ public class SimpleProfiler implements Profiler {
         if (space.columns.size() == 1) {
           nullCount = space.nullCount;
           valueSet = ImmutableSortedSet.copyOf(
-              Iterables.transform(space.values, ONLY));
+              Iterables.transform(space.values, Iterables::getOnlyElement));
         } else {
           nullCount = -1;
           valueSet = null;
@@ -295,7 +278,8 @@ public class SimpleProfiler implements Profiler {
     }
 
     private ImmutableSortedSet<Column> toColumns(Iterable<Integer> ordinals) {
-      return ImmutableSortedSet.copyOf(Iterables.transform(ordinals, get));
+      return ImmutableSortedSet.copyOf(
+          Iterables.transform(ordinals, columns::get));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/RelCollations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelCollations.java b/core/src/main/java/org/apache/calcite/rel/RelCollations.java
index fecf514..c3b16c6 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelCollations.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelCollations.java
@@ -20,7 +20,6 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -40,7 +39,7 @@ public class RelCollations {
    */
   public static final RelCollation EMPTY =
       RelCollationTraitDef.INSTANCE.canonize(
-          new RelCollationImpl(ImmutableList.<RelFieldCollation>of()));
+          new RelCollationImpl(ImmutableList.of()));
 
   /**
    * A collation that cannot be replicated by applying a sort. The only
@@ -131,12 +130,7 @@ public class RelCollations {
   /** Returns the indexes of the fields in a list of field collations. */
   public static List<Integer> ordinals(
       List<RelFieldCollation> fieldCollations) {
-    return Lists.transform(fieldCollations,
-        new Function<RelFieldCollation, Integer>() {
-          public Integer apply(RelFieldCollation input) {
-            return input.getFieldIndex();
-          }
-        });
+    return Lists.transform(fieldCollations, RelFieldCollation::getFieldIndex);
   }
 
   /** Returns whether a collation indicates that the collection is sorted on

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/RelDistributions.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelDistributions.java b/core/src/main/java/org/apache/calcite/rel/RelDistributions.java
index 3c5dc4a..9da83fe 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelDistributions.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelDistributions.java
@@ -24,7 +24,6 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.Mappings;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Ordering;
 
 import java.util.Collection;
@@ -89,7 +88,7 @@ public class RelDistributions {
     private final ImmutableIntList keys;
 
     private RelDistributionImpl(Type type, ImmutableIntList keys) {
-      this.type = Preconditions.checkNotNull(type);
+      this.type = Objects.requireNonNull(type);
       this.keys = ImmutableIntList.copyOf(keys);
       assert type != Type.HASH_DISTRIBUTED
           || keys.size() < 2

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java b/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
index 660b3cd..64f3484 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelFieldCollation.java
@@ -18,8 +18,6 @@ package org.apache.calcite.rel;
 
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 
-import com.google.common.base.Preconditions;
-
 import java.util.Objects;
 
 /**
@@ -209,8 +207,8 @@ public class RelFieldCollation {
       Direction direction,
       NullDirection nullDirection) {
     this.fieldIndex = fieldIndex;
-    this.direction = Preconditions.checkNotNull(direction);
-    this.nullDirection = Preconditions.checkNotNull(nullDirection);
+    this.direction = Objects.requireNonNull(direction);
+    this.nullDirection = Objects.requireNonNull(nullDirection);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/RelRoot.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelRoot.java b/core/src/main/java/org/apache/calcite/rel/RelRoot.java
index a2fa71a..c744492 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelRoot.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelRoot.java
@@ -25,11 +25,11 @@ import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.mapping.Mappings;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 /**
  * Root of a tree of {@link RelNode}.
@@ -92,7 +92,7 @@ public class RelRoot {
     this.validatedRowType = validatedRowType;
     this.kind = kind;
     this.fields = ImmutableList.copyOf(fields);
-    this.collation = Preconditions.checkNotNull(collation);
+    this.collation = Objects.requireNonNull(collation);
   }
 
   /** Creates a simple RelRoot. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
index 149ea1e..84094fc 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/ConverterRule.java
@@ -25,9 +25,8 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.tools.RelBuilderFactory;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
-import com.google.common.base.Predicates;
+import java.util.Objects;
+import java.util.function.Predicate;
 
 /**
  * Abstract base class for a rule which converts from one calling convention to
@@ -51,14 +50,15 @@ public abstract class ConverterRule extends RelOptRule {
    */
   public ConverterRule(Class<? extends RelNode> clazz, RelTrait in,
       RelTrait out, String description) {
-    this(clazz, Predicates.<RelNode>alwaysTrue(), in, out,
+    this(clazz, (Predicate<RelNode>) r -> true, in, out,
         RelFactories.LOGICAL_BUILDER, description);
   }
 
+  @SuppressWarnings("Guava")
   @Deprecated // to be removed before 2.0
   public <R extends RelNode> ConverterRule(Class<R> clazz,
-      Predicate<? super R> predicate, RelTrait in, RelTrait out,
-      String description) {
+      com.google.common.base.Predicate<? super R> predicate,
+      RelTrait in, RelTrait out, String description) {
     this(clazz, predicate, in, out, RelFactories.LOGICAL_BUILDER, description);
   }
 
@@ -80,13 +80,22 @@ public abstract class ConverterRule extends RelOptRule {
         description == null
             ? "ConverterRule<in=" + in + ",out=" + out + ">"
             : description);
-    this.inTrait = Preconditions.checkNotNull(in);
-    this.outTrait = Preconditions.checkNotNull(out);
+    this.inTrait = Objects.requireNonNull(in);
+    this.outTrait = Objects.requireNonNull(out);
 
     // Source and target traits must have same type
     assert in.getTraitDef() == out.getTraitDef();
   }
 
+  @SuppressWarnings("Guava")
+  @Deprecated // to be removed before 2.0
+  public <R extends RelNode> ConverterRule(Class<R> clazz,
+      com.google.common.base.Predicate<? super R> predicate, RelTrait in,
+      RelTrait out, RelBuilderFactory relBuilderFactory, String description) {
+    this(clazz, (Predicate<? super R>) predicate::apply, in, out,
+        relBuilderFactory, description);
+  }
+
   //~ Methods ----------------------------------------------------------------
 
   public Convention getOutConvention() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
index 93f9948..5d699ea 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
@@ -31,7 +31,6 @@ 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.runtime.CalciteException;
-import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.runtime.Resources;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlOperatorBinding;
@@ -45,12 +44,12 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Sets;
 import com.google.common.math.IntMath;
 
+import java.util.HashSet;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -73,26 +72,24 @@ public abstract class Aggregate extends SingleRel {
   /**
    * @see org.apache.calcite.util.Bug#CALCITE_461_FIXED
    */
-  public static final Predicate<Aggregate> IS_SIMPLE =
-      new PredicateImpl<Aggregate>() {
-        public boolean test(Aggregate input) {
-          return input.getGroupType() == Group.SIMPLE;
-        }
-      };
+  public static boolean isSimple(Aggregate aggregate) {
+    return aggregate.getGroupType() == Group.SIMPLE;
+  }
 
-  public static final Predicate<Aggregate> NO_INDICATOR =
-      new PredicateImpl<Aggregate>() {
-        public boolean test(Aggregate input) {
-          return !input.indicator;
-        }
-      };
+  @SuppressWarnings("Guava")
+  @Deprecated // to be converted to Java Predicate before 2.0
+  public static final com.google.common.base.Predicate<Aggregate> IS_SIMPLE =
+      Aggregate::isSimple;
 
-  public static final Predicate<Aggregate> IS_NOT_GRAND_TOTAL =
-      new PredicateImpl<Aggregate>() {
-        public boolean test(Aggregate input) {
-          return input.getGroupCount() > 0;
-        }
-      };
+  @SuppressWarnings("Guava")
+  @Deprecated // to be converted to Java Predicate before 2.0
+  public static final com.google.common.base.Predicate<Aggregate> NO_INDICATOR =
+      Aggregate::noIndicator;
+
+  @SuppressWarnings("Guava")
+  @Deprecated // to be converted to Java Predicate before 2.0
+  public static final com.google.common.base.Predicate<Aggregate>
+      IS_NOT_GRAND_TOTAL = Aggregate::isNotGrandTotal;
 
   //~ Instance fields --------------------------------------------------------
 
@@ -146,7 +143,7 @@ public abstract class Aggregate extends SingleRel {
     super(cluster, traits, child);
     this.indicator = indicator; // true is allowed, but discouraged
     this.aggCalls = ImmutableList.copyOf(aggCalls);
-    this.groupSet = Preconditions.checkNotNull(groupSet);
+    this.groupSet = Objects.requireNonNull(groupSet);
     if (groupSets == null) {
       this.groupSets = ImmutableList.of(groupSet);
     } else {
@@ -165,6 +162,14 @@ public abstract class Aggregate extends SingleRel {
     }
   }
 
+  public static boolean isNotGrandTotal(Aggregate aggregate) {
+    return aggregate.getGroupCount() > 0;
+  }
+
+  public static boolean noIndicator(Aggregate aggregate) {
+    return !aggregate.indicator;
+  }
+
   private boolean isPredicate(RelNode input, int index) {
     final RelDataType type =
         input.getRowType().getFieldList().get(index).getType();
@@ -352,7 +357,7 @@ public abstract class Aggregate extends SingleRel {
     assert groupList.size() == groupSet.cardinality();
     final RelDataTypeFactory.Builder builder = typeFactory.builder();
     final List<RelDataTypeField> fieldList = inputRowType.getFieldList();
-    final Set<String> containedNames = Sets.newHashSet();
+    final Set<String> containedNames = new HashSet<>();
     for (int groupKey : groupList) {
       final RelDataTypeField field = fieldList.get(groupKey);
       containedNames.add(field.getName());

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
index f7194f4..5df2a51 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/AggregateCall.java
@@ -24,7 +24,6 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.mapping.Mapping;
 import org.apache.calcite.util.mapping.Mappings;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -89,9 +88,9 @@ public class AggregateCall {
       int filterArg,
       RelDataType type,
       String name) {
-    this.type = Preconditions.checkNotNull(type);
+    this.type = Objects.requireNonNull(type);
     this.name = name;
-    this.aggFunction = Preconditions.checkNotNull(aggFunction);
+    this.aggFunction = Objects.requireNonNull(aggFunction);
     this.argList = ImmutableList.copyOf(argList);
     this.filterArg = filterArg;
     this.distinct = distinct;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
index 6f218b1..ee7e607 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Correlate.java
@@ -27,7 +27,6 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SemiJoinType;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableBitSet;
@@ -144,7 +143,7 @@ public abstract class Correlate extends BiRel {
       return SqlValidatorUtil.deriveJoinRowType(left.getRowType(),
           right.getRowType(), joinType.toJoinType(),
           getCluster().getTypeFactory(), null,
-          ImmutableList.<RelDataTypeField>of());
+          ImmutableList.of());
     case ANTI:
     case SEMI:
       return left.getRowType();

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
index a45a854..ca5b59d 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/EquiJoin.java
@@ -22,8 +22,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableIntList;
 
-import com.google.common.base.Preconditions;
-
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -39,8 +38,8 @@ public abstract class EquiJoin extends Join {
       ImmutableIntList rightKeys, Set<CorrelationId> variablesSet,
       JoinRelType joinType) {
     super(cluster, traits, left, right, condition, variablesSet, joinType);
-    this.leftKeys = Preconditions.checkNotNull(leftKeys);
-    this.rightKeys = Preconditions.checkNotNull(rightKeys);
+    this.leftKeys = Objects.requireNonNull(leftKeys);
+    this.rightKeys = Objects.requireNonNull(rightKeys);
   }
 
   @Deprecated // to be removed before 2.0

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Exchange.java b/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
index 060fa1c..221740b 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Exchange.java
@@ -30,9 +30,8 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Preconditions;
-
 import java.util.List;
+import java.util.Objects;
 
 /**
  * Relational expression that imposes a particular distribution on its input
@@ -58,7 +57,7 @@ public abstract class Exchange extends SingleRel {
   protected Exchange(RelOptCluster cluster, RelTraitSet traitSet, RelNode input,
       RelDistribution distribution) {
     super(cluster, traitSet, input);
-    this.distribution = Preconditions.checkNotNull(distribution);
+    this.distribution = Objects.requireNonNull(distribution);
 
     assert traitSet.containsIfApplicable(distribution)
         : "traits=" + traitSet + ", distribution" + distribution;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index a9e0441..d20d55e 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -36,12 +36,12 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
 import java.util.Collections;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -101,9 +101,9 @@ public abstract class Join extends BiRel {
       Set<CorrelationId> variablesSet,
       JoinRelType joinType) {
     super(cluster, traitSet, left, right);
-    this.condition = Preconditions.checkNotNull(condition);
+    this.condition = Objects.requireNonNull(condition);
     this.variablesSet = ImmutableSet.copyOf(variablesSet);
-    this.joinType = Preconditions.checkNotNull(joinType);
+    this.joinType = Objects.requireNonNull(joinType);
   }
 
   @Deprecated // to be removed before 2.0

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
index e044913..973734c 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/JoinInfo.java
@@ -25,10 +25,9 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.mapping.IntPair;
 
-import com.google.common.base.Preconditions;
-
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Objects;
 
 /** An analyzed join condition.
  *
@@ -47,8 +46,8 @@ public abstract class JoinInfo {
 
   /** Creates a JoinInfo. */
   protected JoinInfo(ImmutableIntList leftKeys, ImmutableIntList rightKeys) {
-    this.leftKeys = Preconditions.checkNotNull(leftKeys);
-    this.rightKeys = Preconditions.checkNotNull(rightKeys);
+    this.leftKeys = Objects.requireNonNull(leftKeys);
+    this.rightKeys = Objects.requireNonNull(rightKeys);
     assert leftKeys.size() == rightKeys.size();
   }
 
@@ -126,7 +125,7 @@ public abstract class JoinInfo {
     protected NonEquiJoinInfo(ImmutableIntList leftKeys,
         ImmutableIntList rightKeys, RexNode remaining) {
       super(leftKeys, rightKeys);
-      this.remaining = Preconditions.checkNotNull(remaining);
+      this.remaining = Objects.requireNonNull(remaining);
       assert !remaining.isAlwaysTrue();
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Match.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Match.java b/core/src/main/java/org/apache/calcite/rel/core/Match.java
index 2ebafec..7d47bec 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Match.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Match.java
@@ -42,6 +42,7 @@ import com.google.common.collect.ImmutableSortedSet;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeMap;
@@ -98,18 +99,18 @@ public abstract class Match extends SingleRel {
       boolean allRows, List<RexNode> partitionKeys, RelCollation orderKeys,
       RexNode interval) {
     super(cluster, traitSet, input);
-    this.rowType = Preconditions.checkNotNull(rowType);
-    this.pattern = Preconditions.checkNotNull(pattern);
+    this.rowType = Objects.requireNonNull(rowType);
+    this.pattern = Objects.requireNonNull(pattern);
     Preconditions.checkArgument(patternDefinitions.size() > 0);
     this.strictStart = strictStart;
     this.strictEnd = strictEnd;
     this.patternDefinitions = ImmutableMap.copyOf(patternDefinitions);
     this.measures = ImmutableMap.copyOf(measures);
-    this.after = Preconditions.checkNotNull(after);
+    this.after = Objects.requireNonNull(after);
     this.subsets = copyMap(subsets);
     this.allRows = allRows;
     this.partitionKeys = ImmutableList.copyOf(partitionKeys);
-    this.orderKeys = Preconditions.checkNotNull(orderKeys);
+    this.orderKeys = Objects.requireNonNull(orderKeys);
     this.interval = interval;
 
     final AggregateFinder aggregateFinder = new AggregateFinder();

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Project.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index 3119959..b484bf3 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -41,7 +41,6 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.MappingType;
 import org.apache.calcite.util.mapping.Mappings;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -194,13 +193,7 @@ public abstract class Project extends SingleRel {
       return litmus.fail("field names not distinct: {}", rowType);
     }
     //CHECKSTYLE: IGNORE 1
-    if (false && !Util.isDistinct(
-        Lists.transform(exps,
-            new Function<RexNode, Object>() {
-              public Object apply(RexNode a0) {
-                return a0.toString();
-              }
-            }))) {
+    if (false && !Util.isDistinct(Lists.transform(exps, RexNode::toString))) {
       // Projecting the same expression twice is usually a bad idea,
       // because it may create expressions downstream which are equivalent
       // but which look different. We can't ban duplicate projects,

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
index b4ebbca..5870f34 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/RelFactories.java
@@ -35,7 +35,6 @@ import org.apache.calcite.rel.logical.LogicalTableScan;
 import org.apache.calcite.rel.logical.LogicalUnion;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SemiJoinType;
@@ -272,7 +271,7 @@ public class RelFactories {
         RexNode condition, Set<CorrelationId> variablesSet,
         JoinRelType joinType, boolean semiJoinDone) {
       return LogicalJoin.create(left, right, condition, variablesSet, joinType,
-          semiJoinDone, ImmutableList.<RelDataTypeField>of());
+          semiJoinDone, ImmutableList.of());
     }
 
     public RelNode createJoin(RelNode left, RelNode right, RexNode condition,

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
index 433ee5d..774424f 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SemiJoin.java
@@ -25,7 +25,6 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.ImmutableIntList;
@@ -73,7 +72,7 @@ public class SemiJoin extends EquiJoin {
         condition,
         leftKeys,
         rightKeys,
-        ImmutableSet.<CorrelationId>of(),
+        ImmutableSet.of(),
         JoinRelType.INNER);
   }
 
@@ -121,7 +120,7 @@ public class SemiJoin extends EquiJoin {
         JoinRelType.INNER,
         getCluster().getTypeFactory(),
         null,
-        ImmutableList.<RelDataTypeField>of());
+        ImmutableList.of());
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
index 930f23f..eef57c3 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SetOp.java
@@ -28,7 +28,6 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
@@ -102,12 +101,8 @@ public abstract class SetOp extends AbstractRelNode {
   }
 
   @Override protected RelDataType deriveRowType() {
-    final List<RelDataType> inputRowTypes = Lists.transform(inputs,
-        new Function<RelNode, RelDataType>() {
-          public RelDataType apply(RelNode input) {
-            return input.getRowType();
-          }
-        });
+    final List<RelDataType> inputRowTypes =
+        Lists.transform(inputs, RelNode::getRowType);
     final RelDataType rowType =
         getCluster().getTypeFactory().leastRestrictive(inputRowTypes);
     if (rowType == null) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/SortExchange.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/SortExchange.java b/core/src/main/java/org/apache/calcite/rel/core/SortExchange.java
index b6c277a..1854674 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/SortExchange.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/SortExchange.java
@@ -26,7 +26,7 @@ import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 
-import com.google.common.base.Preconditions;
+import java.util.Objects;
 
 /**
  * Relational expression that performs {@link Exchange} and {@link Sort}
@@ -57,7 +57,7 @@ public abstract class SortExchange extends Exchange {
   protected SortExchange(RelOptCluster cluster, RelTraitSet traitSet,
       RelNode input, RelDistribution distribution, RelCollation collation) {
     super(cluster, traitSet, input, distribution);
-    this.collation = Preconditions.checkNotNull(collation);
+    this.collation = Objects.requireNonNull(collation);
 
     assert traitSet.containsIfApplicable(collation)
         : "traits=" + traitSet + ", collation=" + collation;

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
index 7482c39..cdedbd0 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableFunctionScan.java
@@ -95,7 +95,7 @@ public abstract class TableFunctionScan extends AbstractRelNode {
         input.getCluster(), input.getTraitSet(), input.getInputs(),
         input.getExpression("invocation"), (Type) input.get("elementType"),
         input.getRowType("rowType"),
-        ImmutableSet.<RelColumnMapping>of());
+        ImmutableSet.of());
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
index 9dfed22..1fc56e6 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/TableModify.java
@@ -36,6 +36,7 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import com.google.common.base.Preconditions;
 
 import java.util.List;
+import java.util.Objects;
 
 /**
  * Relational expression that modifies a table.
@@ -116,8 +117,8 @@ public abstract class TableModify extends SingleRel {
     this.updateColumnList = updateColumnList;
     this.sourceExpressionList = sourceExpressionList;
     if (operation == Operation.UPDATE) {
-      Preconditions.checkNotNull(updateColumnList);
-      Preconditions.checkNotNull(sourceExpressionList);
+      Objects.requireNonNull(updateColumnList);
+      Objects.requireNonNull(sourceExpressionList);
       Preconditions.checkArgument(sourceExpressionList.size()
           == updateColumnList.size());
     } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/core/Values.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Values.java b/core/src/main/java/org/apache/calcite/rel/core/Values.java
index 30d366d..91c6323 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Values.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Values.java
@@ -28,63 +28,33 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexLiteral;
-import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Pair;
 
-import com.google.common.base.Function;
-import com.google.common.base.Predicate;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.util.List;
+import java.util.function.Predicate;
 
 /**
  * Relational expression whose value is a sequence of zero or more literal row
  * values.
  */
 public abstract class Values extends AbstractRelNode {
-  /**
-   * Lambda that helps render tuples as strings.
-   */
-  private static final Function<ImmutableList<RexLiteral>, Object> F =
-      new Function<ImmutableList<RexLiteral>, Object>() {
-        public Object apply(ImmutableList<RexLiteral> tuple) {
-          String s = tuple.toString();
-          assert s.startsWith("[");
-          assert s.endsWith("]");
-          return "{ " + s.substring(1, s.length() - 1) + " }";
-        }
-      };
 
-  /** Predicate, to be used when defining an operand of a {@link RelOptRule},
-   * that returns true if a Values contains zero tuples.
-   *
-   * <p>This is the conventional way to represent an empty relational
-   * expression. There are several rules that recognize empty relational
-   * expressions and prune away that section of the tree.
-   */
-  public static final Predicate<? super Values> IS_EMPTY =
-      new PredicateImpl<Values>() {
-        public boolean test(Values values) {
-          return values.getTuples().isEmpty();
-        }
-      };
+  public static final Predicate<? super Values> IS_EMPTY_J = Values::isEmpty;
 
-  /** Predicate, to be used when defining an operand of a {@link RelOptRule},
-   * that returns true if a Values contains one or more tuples.
-   *
-   * <p>This is the conventional way to represent an empty relational
-   * expression. There are several rules that recognize empty relational
-   * expressions and prune away that section of the tree.
-   */
-  public static final Predicate<? super Values> IS_NOT_EMPTY =
-      new PredicateImpl<Values>() {
-        public boolean test(Values values) {
-          return !values.getTuples().isEmpty();
-        }
-      };
+  @SuppressWarnings("Guava")
+  @Deprecated // to be removed before 2.0
+  public static final com.google.common.base.Predicate<? super Values>
+      IS_EMPTY = Values::isEmpty;
+
+  @SuppressWarnings("Guava")
+  @Deprecated // to be removed before 2.0
+  public static final com.google.common.base.Predicate<? super Values>
+      IS_NOT_EMPTY = Values::isNotEmpty;
 
   //~ Instance fields --------------------------------------------------------
 
@@ -124,8 +94,40 @@ public abstract class Values extends AbstractRelNode {
         input.getTuples("tuples"), input.getTraitSet());
   }
 
+  /**
+   * Helps render tuples as strings.
+   */
+  private static Object apply(ImmutableList<RexLiteral> tuple) {
+    String s = tuple.toString();
+    assert s.startsWith("[");
+    assert s.endsWith("]");
+    return "{ " + s.substring(1, s.length() - 1) + " }";
+  }
+
   //~ Methods ----------------------------------------------------------------
 
+  /** Predicate, to be used when defining an operand of a {@link RelOptRule},
+   * that returns true if a Values contains zero tuples.
+   *
+   * <p>This is the conventional way to represent an empty relational
+   * expression. There are several rules that recognize empty relational
+   * expressions and prune away that section of the tree.
+   */
+  public static boolean isEmpty(Values values) {
+    return values.getTuples().isEmpty();
+  }
+
+  /** Predicate, to be used when defining an operand of a {@link RelOptRule},
+   * that returns true if a Values contains one or more tuples.
+   *
+   * <p>This is the conventional way to represent an empty relational
+   * expression. There are several rules that recognize empty relational
+   * expressions and prune away that section of the tree.
+   */
+  public static boolean isNotEmpty(Values values) {
+    return !isEmpty(values);
+  }
+
   public ImmutableList<ImmutableList<RexLiteral>> getTuples(RelInput input) {
     return input.getTuples("tuples");
   }
@@ -188,7 +190,7 @@ public abstract class Values extends AbstractRelNode {
         .itemIf("type", rowType,
             pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES)
         .itemIf("type", rowType.getFieldList(), pw.nest())
-        .itemIf("tuples", Lists.transform(tuples, F), !pw.nest())
+        .itemIf("tuples", Lists.transform(tuples, Values::apply), !pw.nest())
         .itemIf("tuples", tuples, pw.nest());
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
index 0d37710..263010e 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalCalc.java
@@ -22,7 +22,6 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Calc;
@@ -36,8 +35,6 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Supplier;
-
 import java.util.List;
 import java.util.Set;
 
@@ -95,17 +92,9 @@ public final class LogicalCalc extends Calc {
     final RelTraitSet traitSet = cluster.traitSet()
         .replace(Convention.NONE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
-            new Supplier<List<RelCollation>>() {
-              public List<RelCollation> get() {
-                return RelMdCollation.calc(mq, input, program);
-              }
-            })
+            () -> RelMdCollation.calc(mq, input, program))
         .replaceIf(RelDistributionTraitDef.INSTANCE,
-            new Supplier<RelDistribution>() {
-              public RelDistribution get() {
-                return RelMdDistribution.calc(mq, input, program);
-              }
-            });
+            () -> RelMdDistribution.calc(mq, input, program));
     return new LogicalCalc(cluster, traitSet, input, program);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
index fce9136..3b5febf 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalFilter.java
@@ -19,9 +19,7 @@ package org.apache.calcite.rel.logical;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
-import org.apache.calcite.rel.RelDistribution;
 import org.apache.calcite.rel.RelDistributionTraitDef;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -34,11 +32,9 @@ import org.apache.calcite.rel.metadata.RelMdDistribution;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableSet;
 
-import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -69,7 +65,7 @@ public final class LogicalFilter extends Filter {
       RexNode condition,
       ImmutableSet<CorrelationId> variablesSet) {
     super(cluster, traitSet, child, condition);
-    this.variablesSet = Preconditions.checkNotNull(variablesSet);
+    this.variablesSet = Objects.requireNonNull(variablesSet);
   }
 
   @Deprecated // to be removed before 2.0
@@ -78,7 +74,7 @@ public final class LogicalFilter extends Filter {
       RelTraitSet traitSet,
       RelNode child,
       RexNode condition) {
-    this(cluster, traitSet, child, condition, ImmutableSet.<CorrelationId>of());
+    this(cluster, traitSet, child, condition, ImmutableSet.of());
   }
 
   @Deprecated // to be removed before 2.0
@@ -87,7 +83,7 @@ public final class LogicalFilter extends Filter {
       RelNode child,
       RexNode condition) {
     this(cluster, cluster.traitSetOf(Convention.NONE), child, condition,
-        ImmutableSet.<CorrelationId>of());
+        ImmutableSet.of());
   }
 
   /**
@@ -100,7 +96,7 @@ public final class LogicalFilter extends Filter {
 
   /** Creates a LogicalFilter. */
   public static LogicalFilter create(final RelNode input, RexNode condition) {
-    return create(input, condition, ImmutableSet.<CorrelationId>of());
+    return create(input, condition, ImmutableSet.of());
   }
 
   /** Creates a LogicalFilter. */
@@ -110,17 +106,9 @@ public final class LogicalFilter extends Filter {
     final RelMetadataQuery mq = cluster.getMetadataQuery();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
         .replaceIfs(RelCollationTraitDef.INSTANCE,
-            new Supplier<List<RelCollation>>() {
-              public List<RelCollation> get() {
-                return RelMdCollation.filter(mq, input);
-              }
-            })
+            () -> RelMdCollation.filter(mq, input))
         .replaceIf(RelDistributionTraitDef.INSTANCE,
-            new Supplier<RelDistribution>() {
-              public RelDistribution get() {
-                return RelMdDistribution.filter(mq, input);
-              }
-            });
+            () -> RelMdDistribution.filter(mq, input));
     return new LogicalFilter(cluster, traitSet, input, condition, variablesSet);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
index aacd72a..696d0b6 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalJoin.java
@@ -29,11 +29,11 @@ import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
 
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 /**
@@ -97,7 +97,7 @@ public final class LogicalJoin extends Join {
       ImmutableList<RelDataTypeField> systemFieldList) {
     super(cluster, traitSet, left, right, condition, variablesSet, joinType);
     this.semiJoinDone = semiJoinDone;
-    this.systemFieldList = Preconditions.checkNotNull(systemFieldList);
+    this.systemFieldList = Objects.requireNonNull(systemFieldList);
   }
 
   @Deprecated // to be removed before 2.0
@@ -115,7 +115,7 @@ public final class LogicalJoin extends Join {
       RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
     this(cluster, cluster.traitSetOf(Convention.NONE), left, right, condition,
         CorrelationId.setOf(variablesStopped), joinType, false,
-        ImmutableList.<RelDataTypeField>of());
+        ImmutableList.of());
   }
 
   @Deprecated // to be removed before 2.0
@@ -133,9 +133,9 @@ public final class LogicalJoin extends Join {
   public LogicalJoin(RelInput input) {
     this(input.getCluster(), input.getCluster().traitSetOf(Convention.NONE),
         input.getInputs().get(0), input.getInputs().get(1),
-        input.getExpression("condition"), ImmutableSet.<CorrelationId>of(),
+        input.getExpression("condition"), ImmutableSet.of(),
         input.getEnum("joinType", JoinRelType.class), false,
-        ImmutableList.<RelDataTypeField>of());
+        ImmutableList.of());
   }
 
   /** Creates a LogicalJoin, flagged with whether it has been translated to a
@@ -161,14 +161,14 @@ public final class LogicalJoin extends Join {
   public static LogicalJoin create(RelNode left, RelNode right,
       RexNode condition, Set<CorrelationId> variablesSet, JoinRelType joinType) {
     return create(left, right, condition, variablesSet, joinType, false,
-        ImmutableList.<RelDataTypeField>of());
+        ImmutableList.of());
   }
 
   @Deprecated // to be removed before 2.0
   public static LogicalJoin create(RelNode left, RelNode right,
       RexNode condition, JoinRelType joinType, Set<String> variablesStopped) {
     return create(left, right, condition, CorrelationId.setOf(variablesStopped),
-        joinType, false, ImmutableList.<RelDataTypeField>of());
+        joinType, false, ImmutableList.of());
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
index 9eec20b..c5bbd4e 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalProject.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel.logical;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelCollations;
 import org.apache.calcite.rel.RelInput;
@@ -34,8 +33,6 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Supplier;
-
 import java.util.List;
 
 /**
@@ -110,13 +107,8 @@ public final class LogicalProject extends Project {
     final RelMetadataQuery mq = cluster.getMetadataQuery();
     final RelTraitSet traitSet =
         cluster.traitSet().replace(Convention.NONE)
-            .replaceIfs(
-                RelCollationTraitDef.INSTANCE,
-                new Supplier<List<RelCollation>>() {
-                  public List<RelCollation> get() {
-                    return RelMdCollation.project(mq, input, projects);
-                  }
-                });
+            .replaceIfs(RelCollationTraitDef.INSTANCE,
+                () -> RelMdCollation.project(mq, input, projects));
     return new LogicalProject(cluster, traitSet, input, projects, rowType);
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
index abad97d..6b6b74a 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalTableScan.java
@@ -20,14 +20,12 @@ import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptTable;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.schema.Table;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -99,15 +97,12 @@ public final class LogicalTableScan extends TableScan {
     final Table table = relOptTable.unwrap(Table.class);
     final RelTraitSet traitSet =
         cluster.traitSetOf(Convention.NONE)
-            .replaceIfs(RelCollationTraitDef.INSTANCE,
-                new Supplier<List<RelCollation>>() {
-                  public List<RelCollation> get() {
-                    if (table != null) {
-                      return table.getStatistic().getCollations();
-                    }
-                    return ImmutableList.of();
-                  }
-                });
+            .replaceIfs(RelCollationTraitDef.INSTANCE, () -> {
+              if (table != null) {
+                return table.getStatistic().getCollations();
+              }
+              return ImmutableList.of();
+            });
     return new LogicalTableScan(cluster, traitSet, relOptTable);
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
index b2934f5..5fc9548 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalValues.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel.logical;
 import org.apache.calcite.plan.Convention;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollationTraitDef;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -31,7 +30,6 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.sql.type.SqlTypeName;
 
-import com.google.common.base.Supplier;
 import com.google.common.collect.ImmutableList;
 
 import java.math.BigDecimal;
@@ -84,12 +82,8 @@ public class LogicalValues extends Values {
       final ImmutableList<ImmutableList<RexLiteral>> tuples) {
     final RelMetadataQuery mq = cluster.getMetadataQuery();
     final RelTraitSet traitSet = cluster.traitSetOf(Convention.NONE)
-        .replaceIfs(
-            RelCollationTraitDef.INSTANCE, new Supplier<List<RelCollation>>() {
-              public List<RelCollation> get() {
-                return RelMdCollation.values(mq, rowType, tuples);
-              }
-            });
+        .replaceIfs(RelCollationTraitDef.INSTANCE,
+            () -> RelMdCollation.values(mq, rowType, tuples));
     return new LogicalValues(cluster, traitSet, rowType, tuples);
   }
 
@@ -103,7 +97,7 @@ public class LogicalValues extends Values {
   public static LogicalValues createEmpty(RelOptCluster cluster,
       RelDataType rowType) {
     return create(cluster, rowType,
-        ImmutableList.<ImmutableList<RexLiteral>>of());
+        ImmutableList.of());
   }
 
   /** Creates a LogicalValues that outputs one row and one column. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
index 987bf5f..f03cc2f 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -24,7 +24,6 @@ import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Window;
 import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rex.RexFieldCollation;
 import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexLocalRef;
@@ -34,7 +33,6 @@ import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
-import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Litmus;
@@ -181,8 +179,7 @@ public final class LogicalWindow extends Window {
     // each window.
     final List<Window.RexWinAggCall> flattenedAggCallList = new ArrayList<>();
     final List<Map.Entry<String, RelDataType>> fieldList =
-        new ArrayList<Map.Entry<String, RelDataType>>(
-            child.getRowType().getFieldList());
+        new ArrayList<>(child.getRowType().getFieldList());
     final int offset = fieldList.size();
 
     // Use better field names for agg calls that are projected.
@@ -349,13 +346,10 @@ public final class LogicalWindow extends Window {
     RelCollation orderKeys = getCollation(
         Lists.newArrayList(
             Iterables.filter(aggWindow.orderKeys,
-              new PredicateImpl<RexFieldCollation>() {
-                public boolean test(RexFieldCollation rexFieldCollation) {
-                  // If ORDER BY references constant (i.e. RexInputRef),
-                  // then we can ignore such ORDER BY key.
-                  return rexFieldCollation.left instanceof RexLocalRef;
-                }
-              })));
+                rexFieldCollation ->
+                    // If ORDER BY references constant (i.e. RexInputRef),
+                    // then we can ignore such ORDER BY key.
+                    rexFieldCollation.left instanceof RexLocalRef)));
     ImmutableBitSet groupSet =
         ImmutableBitSet.of(getProjectOrdinals(aggWindow.partitionKeys));
     final int groupLength = groupSet.length();

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
index f79b187..fd44e47 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel.metadata;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.RelNode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Multimap;
 
@@ -30,6 +29,7 @@ import java.lang.reflect.Proxy;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Implementation of the {@link RelMetadataProvider}
@@ -66,14 +66,12 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
 
     // TODO jvs 30-Mar-2006: Use meta-metadata to decide which metadata
     // query results can stay fresh until the next Ice Age.
-    return new UnboundMetadata<M>() {
-      public M bind(RelNode rel, RelMetadataQuery mq) {
-        final Metadata metadata = function.bind(rel, mq);
-        return metadataClass.cast(
-            Proxy.newProxyInstance(metadataClass.getClassLoader(),
-                new Class[]{metadataClass},
-                new CachingInvocationHandler(metadata)));
-      }
+    return (rel, mq) -> {
+      final Metadata metadata = function.bind(rel, mq);
+      return metadataClass.cast(
+          Proxy.newProxyInstance(metadataClass.getClassLoader(),
+              new Class[]{metadataClass},
+              new CachingInvocationHandler(metadata)));
     };
   }
 
@@ -102,7 +100,7 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
     private final Metadata metadata;
 
     CachingInvocationHandler(Metadata metadata) {
-      this.metadata = Preconditions.checkNotNull(metadata);
+      this.metadata = Objects.requireNonNull(metadata);
     }
 
     public Object invoke(Object proxy, Method method, Object[] args)

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
index 9f1b0cb..af01813 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
@@ -21,7 +21,6 @@ import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultimap;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Multimap;
 
 import java.lang.reflect.InvocationHandler;
@@ -86,20 +85,18 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
     case 1:
       return functions.get(0);
     default:
-      return new UnboundMetadata<M>() {
-        public M bind(RelNode rel, RelMetadataQuery mq) {
-          final List<Metadata> metadataList = Lists.newArrayList();
-          for (UnboundMetadata<M> function : functions) {
-            final Metadata metadata = function.bind(rel, mq);
-            if (metadata != null) {
-              metadataList.add(metadata);
-            }
+      return (rel, mq) -> {
+        final List<Metadata> metadataList = new ArrayList<>();
+        for (UnboundMetadata<M> function : functions) {
+          final Metadata metadata = function.bind(rel, mq);
+          if (metadata != null) {
+            metadataList.add(metadata);
           }
-          return metadataClass.cast(
-              Proxy.newProxyInstance(metadataClass.getClassLoader(),
-                  new Class[]{metadataClass},
-                  new ChainedInvocationHandler(metadataList)));
         }
+        return metadataClass.cast(
+            Proxy.newProxyInstance(metadataClass.getClassLoader(),
+                new Class[]{metadataClass},
+                new ChainedInvocationHandler(metadataList)));
       };
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
index fe937ed..e3e4407 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
@@ -82,7 +82,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.CopyOnWriteArraySet;
 import java.util.concurrent.ExecutionException;
-import javax.annotation.Nonnull;
 
 /**
  * Implementation of the {@link RelMetadataProvider} interface that generates
@@ -102,17 +101,14 @@ public class JaninoRelMetadataProvider implements RelMetadataProvider {
   /** Cache of pre-generated handlers by provider and kind of metadata.
    * For the cache to be effective, providers should implement identity
    * correctly. */
+  @SuppressWarnings("unchecked")
   private static final LoadingCache<Key, MetadataHandler> HANDLERS =
       maxSize(CacheBuilder.newBuilder(),
           SaffronProperties.INSTANCE.metadataHandlerCacheMaximumSize().get())
           .build(
-              new CacheLoader<Key, MetadataHandler>() {
-                public MetadataHandler load(@Nonnull Key key) {
-                  //noinspection unchecked
-                  return load3(key.def, key.provider.handlers(key.def),
-                      key.relClasses);
-                }
-              });
+              CacheLoader.from(key ->
+                  load3(key.def, key.provider.handlers(key.def),
+                      key.relClasses)));
 
   // Pre-register the most common relational operators, to reduce the number of
   // times we re-generate.

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
index 063cf6e..781c697 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
@@ -36,12 +36,7 @@ import java.util.concurrent.ExecutionException;
  */
 public class MetadataFactoryImpl implements MetadataFactory {
   @SuppressWarnings("unchecked")
-  public static final UnboundMetadata<Metadata> DUMMY =
-      new UnboundMetadata<Metadata>() {
-        public Metadata bind(RelNode rel, RelMetadataQuery mq) {
-          return null;
-        }
-      };
+  public static final UnboundMetadata<Metadata> DUMMY = (rel, mq) -> null;
 
   private final LoadingCache<
       Pair<Class<RelNode>, Class<Metadata>>, UnboundMetadata<Metadata>> cache;
@@ -52,16 +47,12 @@ public class MetadataFactoryImpl implements MetadataFactory {
 
   private static CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
       UnboundMetadata<Metadata>> loader(final RelMetadataProvider provider) {
-    return new CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
-        UnboundMetadata<Metadata>>() {
-      @Override public UnboundMetadata<Metadata> load(
-          Pair<Class<RelNode>, Class<Metadata>> key) throws Exception {
-        final UnboundMetadata<Metadata> function =
-            provider.apply(key.left, key.right);
-        // Return DUMMY, not null, so the cache knows to not ask again.
-        return function != null ? function : DUMMY;
-      }
-    };
+    return CacheLoader.from(key -> {
+      final UnboundMetadata<Metadata> function =
+          provider.apply(key.left, key.right);
+      // Return DUMMY, not null, so the cache knows to not ask again.
+      return function != null ? function : DUMMY;
+    });
   }
 
   public <M extends Metadata> M query(RelNode rel, RelMetadataQuery mq,

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
index 9b5abec..20185b7 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
@@ -25,12 +25,10 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.ReflectiveVisitor;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
 
-import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -42,6 +40,7 @@ import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
+import java.util.Objects;
 import java.util.Set;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -130,80 +129,68 @@ public class ReflectiveRelMetadataProvider
         builder.add(space.find(key, method));
       }
       final List<Method> handlerMethods = builder.build();
-      final UnboundMetadata function =
-          new UnboundMetadata() {
-            public Metadata bind(final RelNode rel,
-                final RelMetadataQuery mq) {
-              return (Metadata) Proxy.newProxyInstance(
-                  space.metadataClass0.getClassLoader(),
-                  new Class[]{space.metadataClass0},
-                  new InvocationHandler() {
-                    public Object invoke(Object proxy, Method method,
-                        Object[] args) throws Throwable {
-                      // Suppose we are an implementation of Selectivity
-                      // that wraps "filter", a LogicalFilter. Then we
-                      // implement
-                      //   Selectivity.selectivity(rex)
-                      // by calling method
-                      //   new SelectivityImpl().selectivity(filter, rex)
-                      if (method.equals(
-                          BuiltInMethod.METADATA_REL.method)) {
-                        return rel;
-                      }
-                      if (method.equals(
-                          BuiltInMethod.OBJECT_TO_STRING.method)) {
-                        return space.metadataClass0.getSimpleName() + "(" + rel
-                            + ")";
-                      }
-                      int i = methods.indexOf(method);
-                      if (i < 0) {
-                        throw new AssertionError("not handled: " + method
-                            + " for " + rel);
-                      }
-                      final Method handlerMethod = handlerMethods.get(i);
-                      if (handlerMethod == null) {
-                        throw new AssertionError("not handled: " + method
-                            + " for " + rel);
-                      }
-                      final Object[] args1;
-                      final List key;
-                      if (args == null) {
-                        args1 = new Object[]{rel, mq};
-                        key = FlatLists.of(rel, method);
-                      } else {
-                        args1 = new Object[args.length + 2];
-                        args1[0] = rel;
-                        args1[1] = mq;
-                        System.arraycopy(args, 0, args1, 2, args.length);
+      final UnboundMetadata function = (rel, mq) ->
+          (Metadata) Proxy.newProxyInstance(
+              space.metadataClass0.getClassLoader(),
+              new Class[]{space.metadataClass0}, (proxy, method, args) -> {
+                // Suppose we are an implementation of Selectivity
+                // that wraps "filter", a LogicalFilter. Then we
+                // implement
+                //   Selectivity.selectivity(rex)
+                // by calling method
+                //   new SelectivityImpl().selectivity(filter, rex)
+                if (method.equals(BuiltInMethod.METADATA_REL.method)) {
+                  return rel;
+                }
+                if (method.equals(BuiltInMethod.OBJECT_TO_STRING.method)) {
+                  return space.metadataClass0.getSimpleName() + "(" + rel + ")";
+                }
+                int i = methods.indexOf(method);
+                if (i < 0) {
+                  throw new AssertionError("not handled: " + method
+                      + " for " + rel);
+                }
+                final Method handlerMethod = handlerMethods.get(i);
+                if (handlerMethod == null) {
+                  throw new AssertionError("not handled: " + method
+                      + " for " + rel);
+                }
+                final Object[] args1;
+                final List key1;
+                if (args == null) {
+                  args1 = new Object[]{rel, mq};
+                  key1 = FlatLists.of(rel, method);
+                } else {
+                  args1 = new Object[args.length + 2];
+                  args1[0] = rel;
+                  args1[1] = mq;
+                  System.arraycopy(args, 0, args1, 2, args.length);
 
-                        final Object[] args2 = args1.clone();
-                        args2[1] = method; // replace RelMetadataQuery with method
-                        for (int j = 0; j < args2.length; j++) {
-                          if (args2[j] == null) {
-                            args2[j] = NullSentinel.INSTANCE;
-                          } else if (args2[j] instanceof RexNode) {
-                            // Can't use RexNode.equals - it is not deep
-                            args2[j] = args2[j].toString();
-                          }
-                        }
-                        key = FlatLists.copyOf(args2);
-                      }
-                      if (mq.map.put(key, NullSentinel.INSTANCE) != null) {
-                        throw CyclicMetadataException.INSTANCE;
-                      }
-                      try {
-                        return handlerMethod.invoke(target, args1);
-                      } catch (InvocationTargetException
-                          | UndeclaredThrowableException e) {
-                        Util.throwIfUnchecked(e.getCause());
-                        throw new RuntimeException(e.getCause());
-                      } finally {
-                        mq.map.remove(key);
-                      }
+                  final Object[] args2 = args1.clone();
+                  args2[1] = method; // replace RelMetadataQuery with method
+                  for (int j = 0; j < args2.length; j++) {
+                    if (args2[j] == null) {
+                      args2[j] = NullSentinel.INSTANCE;
+                    } else if (args2[j] instanceof RexNode) {
+                      // Can't use RexNode.equals - it is not deep
+                      args2[j] = args2[j].toString();
                     }
-                  });
-            }
-          };
+                  }
+                  key1 = FlatLists.copyOf(args2);
+                }
+                if (mq.map.put(key1, NullSentinel.INSTANCE) != null) {
+                  throw CyclicMetadataException.INSTANCE;
+                }
+                try {
+                  return handlerMethod.invoke(target, args1);
+                } catch (InvocationTargetException
+                    | UndeclaredThrowableException e) {
+                  Util.throwIfUnchecked(e.getCause());
+                  throw new RuntimeException(e.getCause());
+                } finally {
+                  mq.map.remove(key1);
+                }
+              });
       methodsMap.put(key, function);
     }
     return new ReflectiveRelMetadataProvider(methodsMap, space.metadataClass0,
@@ -313,7 +300,7 @@ public class ReflectiveRelMetadataProvider
      * {@code map}. */
     @SuppressWarnings({ "unchecked", "SuspiciousMethodCalls" })
     Method find(final Class<? extends RelNode> relNodeClass, Method method) {
-      Preconditions.checkNotNull(relNodeClass);
+      Objects.requireNonNull(relNodeClass);
       for (Class r = relNodeClass;;) {
         Method implementingMethod = handlerMap.get(Pair.of(r, method));
         if (implementingMethod != null) {