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:17 UTC

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

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java b/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
new file mode 100644
index 0000000..c727441
--- /dev/null
+++ b/elasticsearch5/src/test/java/org/apache/calcite/test/Elasticsearch5AdapterIT.java
@@ -0,0 +1,265 @@
+/*
+ * 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.test;
+
+import org.apache.calcite.util.Util;
+
+import com.google.common.collect.ImmutableMap;
+
+import org.junit.Test;
+
+import java.util.List;
+import java.util.function.Consumer;
+
+/**
+ * Tests for the {@code org.apache.calcite.adapter.elasticsearch} package.
+ *
+ * <p>Before calling this test, you need to populate Elasticsearch, as follows:
+ *
+ * <blockquote><code>
+ * git clone https://github.com/vlsi/calcite-test-dataset<br>
+ * cd calcite-test-dataset<br>
+ * mvn install
+ * </code></blockquote>
+ *
+ * <p>This will create a virtual machine with Elasticsearch and the "zips" test
+ * dataset.
+ */
+public class Elasticsearch5AdapterIT {
+  /**
+   * Whether to run Elasticsearch tests. Enabled by default, however test is only
+   * included if "it" profile is activated ({@code -Pit}). To disable,
+   * specify {@code -Dcalcite.test.elasticsearch=false} on the Java command line.
+   */
+  private static final boolean ENABLED = Util.getBooleanProperty("calcite.test.elasticsearch",
+      true);
+
+  /** Connection factory based on the "zips-es" model. */
+  private static final ImmutableMap<String, String> ZIPS = ImmutableMap.of("model",
+      Elasticsearch5AdapterIT.class.getResource("/elasticsearch-zips-model.json").getPath());
+
+  /** Whether to run this test. */
+  private boolean enabled() {
+    return ENABLED;
+  }
+
+  /** Returns a function that checks that a particular Elasticsearch pipeline is
+   * generated to implement a query. */
+  private static Consumer<List> elasticsearchChecker(final String... strings) {
+    return actual -> {
+      Object[] actualArray = actual == null || actual.isEmpty() ? null
+          : ((List) actual.get(0)).toArray();
+      CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
+          actualArray);
+    };
+  }
+
+  @Test public void testSort() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], dir0=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips order by \"state\"")
+        .returnsCount(10)
+        .explainContains(explain);
+  }
+
+  @Test public void testSortLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "order by \"state\", \"id\" offset 2 rows fetch next 3 rows only";
+    CalciteAssert.that()
+        .with(ZIPS)
+        .query(sql)
+        .returnsUnordered("state=AK; id=99503",
+            "state=AK; id=99504",
+            "state=AK; id=99505")
+        .queryContains(
+            elasticsearchChecker(
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]",
+                "\"from\": 2",
+                "\"size\": 3"));
+  }
+
+  @Test public void testOffsetLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "offset 2 fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"from\": 2",
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testLimit() {
+    final String sql = "select \"state\", \"id\" from zips\n"
+        + "fetch next 3 rows only";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(
+                "\"size\": 3",
+                "\"fields\" : [\"state\", \"id\"], \"script_fields\": {}"));
+  }
+
+  @Test public void testFilterSort() {
+    final String sql = "select * from zips\n"
+        + "where \"city\" = 'SPRINGFIELD' and \"id\" >= '70000'\n"
+        + "order by \"state\", \"id\"";
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchSort(sort0=[$4], sort1=[$5], dir0=[ASC], dir1=[ASC])\n"
+        + "    ElasticsearchProject(city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], longitude=[CAST(ITEM(ITEM($0, 'loc'), 0)):FLOAT], latitude=[CAST(ITEM(ITEM($0, 'loc'), 1)):FLOAT], pop=[CAST(ITEM($0, 'pop')):INTEGER], state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], id=[CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "      ElasticsearchFilter(condition=[AND(=(CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'SPRINGFIELD'), >=(CAST(ITEM($0, 'id')):VARCHAR(5) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", '70000'))])\n"
+        + "        ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .returnsOrdered(
+            "city=SPRINGFIELD; longitude=-92.54567; latitude=35.274879; pop=752; state=AR; id=72157",
+            "city=SPRINGFIELD; longitude=-102.617322; latitude=37.406727; pop=1992; state=CO; id=81073",
+            "city=SPRINGFIELD; longitude=-90.577479; latitude=30.415738; pop=5597; state=LA; id=70462",
+            "city=SPRINGFIELD; longitude=-123.015259; latitude=44.06106; pop=32384; state=OR; id=97477",
+            "city=SPRINGFIELD; longitude=-122.917108; latitude=44.056056; pop=27521; state=OR; id=97478")
+        .queryContains(
+            elasticsearchChecker("\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                    + "{\"must\":[{\"term\":{\"city\":\"springfield\"}},{\"range\":{\"id\":{\"gte\":\"70000\"}}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}",
+                "\"sort\": [ {\"state\": \"asc\"}, {\"id\": \"asc\"}]"))
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterSortDesc() {
+    final String sql = "select * from zips\n"
+        + "where \"pop\" BETWEEN 20000 AND 20100\n"
+        + "order by \"state\" desc, \"pop\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(4)
+        .returnsOrdered(
+            "city=SHERIDAN; longitude=-106.964795; latitude=44.78486; pop=20025; state=WY; id=82801",
+            "city=MOUNTLAKE TERRAC; longitude=-122.304036; latitude=47.793061; pop=20059; state=WA; id=98043",
+            "city=FALMOUTH; longitude=-77.404537; latitude=38.314557; pop=20039; state=VA; id=22405",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104");
+  }
+
+  @Test public void testFilterRedundant() {
+    final String sql = "select * from zips\n"
+        + "where \"state\" > 'CA' and \"state\" < 'AZ' and \"state\" = 'OK'";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .runs()
+        .queryContains(
+            elasticsearchChecker(""
+                + "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":"
+                + "{\"must\":[{\"term\":{\"state\":\"ok\"}}]}}}}",
+                "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"));
+  }
+
+  @Test public void testInPlan() {
+    final String[] searches = {
+        "\"query\" : {\"constant_score\":{\"filter\":{\"bool\":{\"should\":"
+          + "[{\"bool\":{\"must\":[{\"term\":{\"pop\":20012}}]}},{\"bool\":{\"must\":[{\"term\":"
+          + "{\"pop\":15590}}]}}]}}}}",
+        "\"fields\" : [\"city\", \"pop\", \"state\", \"id\"], \"script_fields\": {\"longitude\":{\"script\":\"_source.loc[0]\"}, \"latitude\":{\"script\":\"_source.loc[1]\"}}"
+    };
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select * from zips where \"pop\" in (20012, 15590)")
+        .returnsUnordered(
+            "city=COVINA; longitude=-117.884285; latitude=34.08596; pop=15590; state=CA; id=91723",
+            "city=ARLINGTON; longitude=-97.091987; latitude=32.654752; pop=15590; state=TX; id=76018",
+            "city=CROFTON; longitude=-76.680166; latitude=39.011163; pop=15590; state=MD; id=21114",
+            "city=FORT WORTH; longitude=-97.318409; latitude=32.725551; pop=20012; state=TX; id=76104",
+            "city=DINUBA; longitude=-119.39087; latitude=36.534931; pop=20012; state=CA; id=93618")
+        .queryContains(elasticsearchChecker(searches));
+  }
+
+  @Test public void testZips() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips")
+        .returnsCount(10);
+  }
+
+  @Test public void testProject() {
+    final String sql = "select \"state\", \"city\", 0 as \"zero\"\n"
+        + "from zips\n"
+        + "order by \"state\", \"city\"";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query(sql)
+        .limit(2)
+        .returnsUnordered("state=AK; city=ELMENDORF AFB; zero=0",
+            "state=AK; city=EIELSON AFB; zero=0")
+        .queryContains(
+            elasticsearchChecker("\"sort\": [ {\"state\": \"asc\"}, {\"city\": \"asc\"}]",
+                "\"fields\" : [\"state\", \"city\"], \"script_fields\": {\"zero\":{\"script\": \"0\"}}"));
+  }
+
+  @Test public void testFilter() {
+    final String explain = "PLAN=ElasticsearchToEnumerableConverter\n"
+        + "  ElasticsearchProject(state=[CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"], city=[CAST(ITEM($0, 'city')):VARCHAR(20) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\"])\n"
+        + "    ElasticsearchFilter(condition=[=(CAST(ITEM($0, 'state')):VARCHAR(2) CHARACTER SET \"ISO-8859-1\" COLLATE \"ISO-8859-1$en_US$primary\", 'CA')])\n"
+        + "      ElasticsearchTableScan(table=[[elasticsearch_raw, zips]])";
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" = 'CA'")
+        .limit(2)
+        .returnsUnordered("state=CA; city=LOS ANGELES",
+            "state=CA; city=LOS ANGELES")
+        .explainContains(explain);
+  }
+
+  @Test public void testFilterReversed() {
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where 'WI' < \"state\"")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+    CalciteAssert.that()
+        .enable(enabled())
+        .with(ZIPS)
+        .query("select \"state\", \"city\" from zips where \"state\" > 'WI'")
+        .limit(2)
+        .returnsUnordered("state=WV; city=WELCH",
+            "state=WV; city=HANOVER");
+  }
+}
+
+// End Elasticsearch5AdapterIT.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/elasticsearch5/src/test/java/org/apache/calcite/test/ElasticsearchChecker.java
----------------------------------------------------------------------
diff --git a/elasticsearch5/src/test/java/org/apache/calcite/test/ElasticsearchChecker.java b/elasticsearch5/src/test/java/org/apache/calcite/test/ElasticsearchChecker.java
index 550c04c..be236ed 100644
--- a/elasticsearch5/src/test/java/org/apache/calcite/test/ElasticsearchChecker.java
+++ b/elasticsearch5/src/test/java/org/apache/calcite/test/ElasticsearchChecker.java
@@ -16,11 +16,8 @@
  */
 package org.apache.calcite.test;
 
-import com.google.common.base.Function;
-
 import java.util.List;
-
-import javax.annotation.Nullable;
+import java.util.function.Consumer;
 
 /**
  * Utility methods for Elasticsearch tests.
@@ -29,23 +26,18 @@ public class ElasticsearchChecker {
 
   private ElasticsearchChecker() {}
 
-  /**
-   * Returns a function that checks that a particular Elasticsearch pipeline is
+  /** Returns a function that checks that a particular Elasticsearch pipeline is
    * generated to implement a query.
    *
-   * @param strings expected expressions
-   * @return validation function
+   * @param strings list of expected queries
+   * @return function to perform the check
    */
-  public static Function<List, Void> elasticsearchChecker(final String... strings) {
-    return new Function<List, Void>() {
-      @Nullable
-      @Override public Void apply(@Nullable List actual) {
-        Object[] actualArray = actual == null || actual.isEmpty() ? null
-            : ((List) actual.get(0)).toArray();
-        CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
-            actualArray);
-        return null;
-      }
+  public static Consumer<List> elasticsearchChecker(final String... strings) {
+    return actual -> {
+      Object[] actualArray = actual == null || actual.isEmpty() ? null
+          : ((List) actual.get(0)).toArray();
+      CalciteAssert.assertArrayEqual("expected Elasticsearch query not found", strings,
+          actualArray);
     };
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
index afc3601..d42a979 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvEnumerator.java
@@ -331,13 +331,13 @@ class CsvEnumerator<E> implements Enumerator<E> {
     private final boolean stream;
 
     ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields) {
-      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[fieldTypes.size()]);
+      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[0]);
       this.fields = fields;
       this.stream = false;
     }
 
     ArrayRowConverter(List<CsvFieldType> fieldTypes, int[] fields, boolean stream) {
-      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[fieldTypes.size()]);
+      this.fieldTypes = fieldTypes.toArray(new CsvFieldType[0]);
       this.fields = fields;
       this.stream = stream;
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
----------------------------------------------------------------------
diff --git a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
index 7c971f5..8744966 100644
--- a/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
+++ b/example/csv/src/main/java/org/apache/calcite/adapter/csv/CsvSchema.java
@@ -24,7 +24,6 @@ import org.apache.calcite.util.Sources;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.util.Map;
 
 /**
@@ -77,14 +76,11 @@ public class CsvSchema extends AbstractSchema {
     // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
     // ".json.gz".
     final Source baseSource = Sources.of(directoryFile);
-    File[] files = directoryFile.listFiles(
-        new FilenameFilter() {
-          public boolean accept(File dir, String name) {
-            final String nameSansGz = trim(name, ".gz");
-            return nameSansGz.endsWith(".csv")
-                || nameSansGz.endsWith(".json");
-          }
-        });
+    File[] files = directoryFile.listFiles((dir, name) -> {
+      final String nameSansGz = trim(name, ".gz");
+      return nameSansGz.endsWith(".csv")
+          || nameSansGz.endsWith(".json");
+    });
     if (files == null) {
       System.out.println("directory " + directoryFile + " not found");
       files = new File[0];

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
----------------------------------------------------------------------
diff --git a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
index b739a9d..f3f5432 100644
--- a/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
+++ b/example/csv/src/test/java/org/apache/calcite/test/CsvTest.java
@@ -23,7 +23,6 @@ import org.apache.calcite.schema.Schema;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Ordering;
 
@@ -54,6 +53,7 @@ import java.util.Properties;
 import java.util.concurrent.ArrayBlockingQueue;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.Callable;
+import java.util.function.Consumer;
 
 import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -194,17 +194,14 @@ public class CsvTest {
     final String sql = "select empno * 3 as e3\n"
         + "from long_emps where empno = 100";
 
-    sql("bug", sql).checking(new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          assertThat(resultSet.next(), is(true));
-          Long o = (Long) resultSet.getObject(1);
-          assertThat(o, is(300L));
-          assertThat(resultSet.next(), is(false));
-          return null;
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
+    sql("bug", sql).checking(resultSet -> {
+      try {
+        assertThat(resultSet.next(), is(true));
+        Long o = (Long) resultSet.getObject(1);
+        assertThat(o, is(300L));
+        assertThat(resultSet.next(), is(false));
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
     }).ok();
   }
@@ -313,60 +310,41 @@ public class CsvTest {
   }
 
   private Fluent sql(String model, String sql) {
-    return new Fluent(model, sql, output());
-  }
-
-  private Function<ResultSet, Void> output() {
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          output(resultSet, System.out);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
-      }
-    };
+    return new Fluent(model, sql, this::output);
   }
 
   /** Returns a function that checks the contents of a result set against an
    * expected string. */
-  private static Function<ResultSet, Void> expect(final String... expected) {
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          final List<String> lines = new ArrayList<>();
-          CsvTest.collect(lines, resultSet);
-          Assert.assertEquals(Arrays.asList(expected), lines);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+  private static Consumer<ResultSet> expect(final String... expected) {
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        CsvTest.collect(lines, resultSet);
+        Assert.assertEquals(Arrays.asList(expected), lines);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
     };
   }
 
   /** Returns a function that checks the contents of a result set against an
    * expected string. */
-  private static Function<ResultSet, Void> expectUnordered(String... expected) {
+  private static Consumer<ResultSet> expectUnordered(String... expected) {
     final List<String> expectedLines =
         Ordering.natural().immutableSortedCopy(Arrays.asList(expected));
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          final List<String> lines = new ArrayList<>();
-          CsvTest.collect(lines, resultSet);
-          Collections.sort(lines);
-          Assert.assertEquals(expectedLines, lines);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        CsvTest.collect(lines, resultSet);
+        Collections.sort(lines);
+        Assert.assertEquals(expectedLines, lines);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
     };
   }
 
-  private void checkSql(String sql, String model, Function<ResultSet, Void> fn)
+  private void checkSql(String sql, String model, Consumer<ResultSet> fn)
       throws SQLException {
     Connection connection = null;
     Statement statement = null;
@@ -378,7 +356,7 @@ public class CsvTest {
       final ResultSet resultSet =
           statement.executeQuery(
               sql);
-      fn.apply(resultSet);
+      fn.accept(resultSet);
     } finally {
       close(connection, statement);
     }
@@ -703,7 +681,7 @@ public class CsvTest {
       final Schema schema =
           CsvSchemaFactory.INSTANCE
               .create(calciteConnection.getRootSchema(), null,
-                  ImmutableMap.<String, Object>of("directory",
+                  ImmutableMap.of("directory",
                       resourcePath("sales"), "flavor", "scannable"));
       calciteConnection.getRootSchema().add("TEST", schema);
       final String sql = "select * from \"TEST\".\"DEPTS\" where \"NAME\" = ?";
@@ -712,8 +690,8 @@ public class CsvTest {
 
       statement2.setString(1, "Sales");
       final ResultSet resultSet1 = statement2.executeQuery();
-      Function<ResultSet, Void> expect = expect("DEPTNO=10; NAME=Sales");
-      expect.apply(resultSet1);
+      Consumer<ResultSet> expect = expect("DEPTNO=10; NAME=Sales");
+      expect.accept(resultSet1);
     }
   }
 
@@ -957,35 +935,38 @@ public class CsvTest {
 
   /** Creates a command that appends a line to the CSV file. */
   private Callable<Void> writeLine(final PrintWriter pw, final String line) {
-    return new Callable<Void>() {
-      @Override public Void call() throws Exception {
-        pw.println(line);
-        pw.flush();
-        return null;
-      }
+    return () -> {
+      pw.println(line);
+      pw.flush();
+      return null;
     };
   }
 
   /** Creates a command that sleeps. */
   private Callable<Void> sleep(final long millis) {
-    return new Callable<Void>() {
-      @Override public Void call() throws Exception {
-        Thread.sleep(millis);
-        return null;
-      }
+    return () -> {
+      Thread.sleep(millis);
+      return null;
     };
   }
 
   /** Creates a command that cancels a statement. */
   private Callable<Void> cancel(final Statement statement) {
-    return new Callable<Void>() {
-      @Override public Void call() throws Exception {
-        statement.cancel();
-        return null;
-      }
+    return () -> {
+      statement.cancel();
+      return null;
     };
   }
 
+  private Void output(ResultSet resultSet) {
+    try {
+      output(resultSet, System.out);
+    } catch (SQLException e) {
+      throw new RuntimeException(e);
+    }
+    return null;
+  }
+
   /** Receives commands on a queue and executes them on its own thread.
    * Call {@link #close} to terminate.
    *
@@ -1003,12 +984,7 @@ public class CsvTest {
     private Exception e;
 
     /** The poison pill command. */
-    final Callable<E> end =
-        new Callable<E>() {
-          public E call() {
-            return null;
-          }
-        };
+    final Callable<E> end = () -> null;
 
     public void run() {
       try {
@@ -1037,9 +1013,9 @@ public class CsvTest {
   private class Fluent {
     private final String model;
     private final String sql;
-    private final Function<ResultSet, Void> expect;
+    private final Consumer<ResultSet> expect;
 
-    Fluent(String model, String sql, Function<ResultSet, Void> expect) {
+    Fluent(String model, String sql, Consumer<ResultSet> expect) {
       this.model = model;
       this.sql = sql;
       this.expect = expect;
@@ -1056,7 +1032,7 @@ public class CsvTest {
     }
 
     /** Assigns a function to call to test whether output is correct. */
-    Fluent checking(Function<ResultSet, Void> expect) {
+    Fluent checking(Consumer<ResultSet> expect) {
       return new Fluent(model, sql, expect);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/example/function/src/main/java/org/apache/calcite/example/maze/Maze.java
----------------------------------------------------------------------
diff --git a/example/function/src/main/java/org/apache/calcite/example/maze/Maze.java b/example/function/src/main/java/org/apache/calcite/example/maze/Maze.java
index eb738fa..d0ee05e 100644
--- a/example/function/src/main/java/org/apache/calcite/example/maze/Maze.java
+++ b/example/function/src/main/java/org/apache/calcite/example/maze/Maze.java
@@ -80,17 +80,11 @@ class Maze {
     final StringBuilder b2 = new StringBuilder();
     final CellContent cellContent;
     if (space) {
-      cellContent = new CellContent() {
-        public String get(int c) {
-          return "  ";
-        }
-      };
+      cellContent = c -> "  ";
     } else {
-      cellContent = new CellContent() {
-        public String get(int c) {
-          String s = region(c) + "";
-          return s.length() == 1 ? " " + s : s;
-        }
+      cellContent = c -> {
+        String s = region(c) + "";
+        return s.length() == 1 ? " " + s : s;
       };
     }
     for (int y = 0; y < height; y++) {
@@ -113,11 +107,7 @@ class Maze {
     if (solutionSet == null) {
       cellContent = CellContent.SPACE;
     } else {
-      cellContent = new CellContent() {
-        public String get(int c) {
-          return solutionSet.contains(c) ? "* " : "  ";
-        }
-      };
+      cellContent = c -> solutionSet.contains(c) ? "* " : "  ";
     }
     return new Enumerator<String>() {
       int i = -1;
@@ -352,11 +342,7 @@ class Maze {
   /** Callback to get what to print in a particular cell. Must be two characters
    * long, usually two spaces. */
   interface CellContent {
-    CellContent SPACE = new CellContent() {
-      public String get(int c) {
-        return "  ";
-      }
-    };
+    CellContent SPACE = c -> "  ";
 
     String get(int c);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/example/function/src/main/java/org/apache/calcite/example/maze/MazeTable.java
----------------------------------------------------------------------
diff --git a/example/function/src/main/java/org/apache/calcite/example/maze/MazeTable.java b/example/function/src/main/java/org/apache/calcite/example/maze/MazeTable.java
index a08b366..767dbb0 100644
--- a/example/function/src/main/java/org/apache/calcite/example/maze/MazeTable.java
+++ b/example/function/src/main/java/org/apache/calcite/example/maze/MazeTable.java
@@ -21,7 +21,6 @@ import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.Enumerable;
 import org.apache.calcite.linq4j.Enumerator;
 import org.apache.calcite.linq4j.Linq4j;
-import org.apache.calcite.linq4j.function.Function1;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.schema.ScannableTable;
@@ -103,11 +102,7 @@ public class MazeTable extends AbstractTable implements ScannableTable {
           solutionSet = null;
         }
         return Linq4j.transform(maze.enumerator(solutionSet),
-            new Function1<String, Object[]>() {
-              public Object[] apply(String s) {
-                return new Object[] {s};
-              }
-            });
+            s -> new Object[] {s});
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
index 3a635e4..3f19e4b 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileRowConverter.java
@@ -21,8 +21,6 @@ import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.util.Pair;
 
-import com.google.common.base.Joiner;
-
 import com.joestelmach.natty.DateGroup;
 import com.joestelmach.natty.Parser;
 
@@ -31,7 +29,6 @@ import org.jsoup.select.Elements;
 
 import java.text.NumberFormat;
 import java.text.ParseException;
-
 import java.util.ArrayList;
 import java.util.HashSet;
 import java.util.LinkedHashMap;
@@ -42,7 +39,6 @@ import java.util.Set;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
 
-
 /**
  * FileRowConverter.
  */
@@ -267,7 +263,7 @@ class FileRowConverter {
         }
       }
 
-      String cellString = Joiner.on(" ").join(cellText).trim();
+      String cellString = String.join(" ", cellText).trim();
 
       // replace
       if (this.replacePattern != null) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
----------------------------------------------------------------------
diff --git a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
index 97c5cb4..29f3b9a 100644
--- a/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
+++ b/file/src/main/java/org/apache/calcite/adapter/file/FileSchema.java
@@ -29,7 +29,6 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 
 import java.io.File;
-import java.io.FilenameFilter;
 import java.net.MalformedURLException;
 import java.util.List;
 import java.util.Map;
@@ -89,14 +88,11 @@ class FileSchema extends AbstractSchema {
     // Look for files in the directory ending in ".csv", ".csv.gz", ".json",
     // ".json.gz".
     final Source baseSource = Sources.of(baseDirectory);
-    File[] files = baseDirectory.listFiles(
-        new FilenameFilter() {
-          public boolean accept(File dir, String name) {
-            final String nameSansGz = trim(name, ".gz");
-            return nameSansGz.endsWith(".csv")
-                || nameSansGz.endsWith(".json");
-          }
-        });
+    File[] files = baseDirectory.listFiles((dir, name) -> {
+      final String nameSansGz = trim(name, ".gz");
+      return nameSansGz.endsWith(".csv")
+          || nameSansGz.endsWith(".json");
+    });
     if (files == null) {
       System.out.println("directory " + baseDirectory + " not found");
       files = new File[0];

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
----------------------------------------------------------------------
diff --git a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java b/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
index 8d31770..3437b69 100644
--- a/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
+++ b/file/src/test/java/org/apache/calcite/adapter/file/SqlTest.java
@@ -18,7 +18,6 @@ package org.apache.calcite.adapter.file;
 
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Ordering;
 
 import org.junit.Assert;
@@ -37,6 +36,7 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Properties;
+import java.util.function.Function;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
@@ -50,34 +50,30 @@ public class SqlTest {
   // helper functions
 
   private Fluent sql(String model, String sql) {
-    return new Fluent(model, sql, new Function<ResultSet, Void>() {
-      public Void apply(ResultSet input) {
-        throw new AssertionError();
-      }
+    return new Fluent(model, sql, input -> {
+      throw new AssertionError();
     });
   }
 
-  private Function<ResultSet, Void> expect(String... expectedLines) {
+  private static Function<ResultSet, Void> expect(String... expectedLines) {
     final StringBuilder b = new StringBuilder();
     for (String s : expectedLines) {
       b.append(s).append('\n');
     }
     final String expected = b.toString();
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          String actual = SqlTest.toString(resultSet);
-          if (!expected.equals(actual)) {
-            System.out.println("Assertion failure:");
-            System.out.println("\tExpected: '" + expected + "'");
-            System.out.println("\tActual: '" + actual + "'");
-          }
-          assertEquals(expected, actual);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
+    return resultSet -> {
+      try {
+        String actual = toString(resultSet);
+        if (!expected.equals(actual)) {
+          System.out.println("Assertion failure:");
+          System.out.println("\tExpected: '" + expected + "'");
+          System.out.println("\tActual: '" + actual + "'");
         }
-        return null;
+        assertEquals(expected, actual);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
+      return null;
     };
   }
 
@@ -86,18 +82,16 @@ public class SqlTest {
   private static Function<ResultSet, Void> expectUnordered(String... expected) {
     final List<String> expectedLines =
         Ordering.natural().immutableSortedCopy(Arrays.asList(expected));
-    return new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          final List<String> lines = new ArrayList<>();
-          SqlTest.collect(lines, resultSet);
-          Collections.sort(lines);
-          Assert.assertEquals(expectedLines, lines);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        SqlTest.collect(lines, resultSet);
+        Collections.sort(lines);
+        Assert.assertEquals(expectedLines, lines);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
+      return null;
     };
   }
 
@@ -259,21 +253,19 @@ public class SqlTest {
    * "EmptyFileHasNoColumns". */
   @Test public void testCsvSalesEmpty() throws SQLException {
     final String sql = "select * from sales.\"EMPTY\"";
-    checkSql(sql, "sales-csv", new Function<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          assertThat(resultSet.getMetaData().getColumnCount(), is(1));
-          assertThat(resultSet.getMetaData().getColumnName(1),
-              is("EmptyFileHasNoColumns"));
-          assertThat(resultSet.getMetaData().getColumnType(1),
-              is(Types.BOOLEAN));
-          String actual = SqlTest.toString(resultSet);
-          assertThat(actual, is(""));
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+    checkSql(sql, "sales-csv", resultSet -> {
+      try {
+        assertThat(resultSet.getMetaData().getColumnCount(), is(1));
+        assertThat(resultSet.getMetaData().getColumnName(1),
+            is("EmptyFileHasNoColumns"));
+        assertThat(resultSet.getMetaData().getColumnType(1),
+            is(Types.BOOLEAN));
+        String actual = toString(resultSet);
+        assertThat(actual, is(""));
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
+      return null;
     });
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeRules.java
----------------------------------------------------------------------
diff --git a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeRules.java b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeRules.java
index bfac3f0..0e006eb 100644
--- a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeRules.java
+++ b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeRules.java
@@ -39,11 +39,10 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 
-import com.google.common.base.Predicate;
-
 import java.util.AbstractList;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.function.Predicate;
 
 /**
  * Rules and relational operators for {@link GeodeRel#CONVENTION} calling convention.
@@ -208,15 +207,11 @@ public class GeodeRules {
 
     private static final GeodeSortLimitRule INSTANCE =
         new GeodeSortLimitRule(
-          new Predicate<Sort>() {
-            public boolean apply(Sort input) {
-              // OQL doesn't support for offsets (e.g. LIMIT 10 OFFSET 500)
-              return input.offset == null;
-            }
-          });
+            // OQL doesn't support for offsets (e.g. LIMIT 10 OFFSET 500)
+            sort -> sort.offset == null);
 
     GeodeSortLimitRule(Predicate<Sort> predicate) {
-      super(operand(Sort.class, null, predicate, any()), "GeodeSortLimitRule");
+      super(operandJ(Sort.class, null, predicate, any()), "GeodeSortLimitRule");
     }
 
     @Override public void onMatch(RelOptRuleCall call) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeSchemaFactory.java
----------------------------------------------------------------------
diff --git a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeSchemaFactory.java b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeSchemaFactory.java
index 8052925..0348a0d 100644
--- a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeSchemaFactory.java
+++ b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeSchemaFactory.java
@@ -57,7 +57,7 @@ public class GeodeSchemaFactory implements SchemaFactory {
     }
 
     if (allowSpatialFunctions) {
-      ModelHandler.addFunctions(parentSchema, null, ImmutableList.<String>of(),
+      ModelHandler.addFunctions(parentSchema, null, ImmutableList.of(),
           GeoFunctions.class.getName(), "*", true);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeToEnumerableConverter.java
----------------------------------------------------------------------
diff --git a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeToEnumerableConverter.java b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeToEnumerableConverter.java
index 5f85907..4b44573 100644
--- a/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeToEnumerableConverter.java
+++ b/geode/src/main/java/org/apache/calcite/adapter/geode/rel/GeodeToEnumerableConverter.java
@@ -39,7 +39,6 @@ import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
 
-import com.google.common.base.Function;
 import com.google.common.collect.Lists;
 
 import java.lang.reflect.Method;
@@ -152,12 +151,7 @@ public class GeodeToEnumerableConverter extends ConverterImpl implements Enumera
    * {@code {ConstantExpression("x"), ConstantExpression("y")}}.
    */
   private static <T> List<Expression> constantList(List<T> values) {
-    return Lists.transform(values,
-        new Function<T, Expression>() {
-          public Expression apply(T a0) {
-            return Expressions.constant(a0);
-          }
-        });
+    return Lists.transform(values, Expressions::constant);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/geode/src/main/java/org/apache/calcite/adapter/geode/util/JavaTypeFactoryExtImpl.java
----------------------------------------------------------------------
diff --git a/geode/src/main/java/org/apache/calcite/adapter/geode/util/JavaTypeFactoryExtImpl.java b/geode/src/main/java/org/apache/calcite/adapter/geode/util/JavaTypeFactoryExtImpl.java
index ca17342..90f57d4 100644
--- a/geode/src/main/java/org/apache/calcite/adapter/geode/util/JavaTypeFactoryExtImpl.java
+++ b/geode/src/main/java/org/apache/calcite/adapter/geode/util/JavaTypeFactoryExtImpl.java
@@ -33,7 +33,6 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-
 /**
  * Implementation of {@link JavaTypeFactory}.
  *

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
----------------------------------------------------------------------
diff --git a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
index cdec337..5e99119 100644
--- a/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
+++ b/geode/src/test/java/org/apache/calcite/adapter/geode/rel/BaseGeodeAdapterIT.java
@@ -43,17 +43,15 @@ public class BaseGeodeAdapterIT {
    * expected string.
    */
   private static Function1<ResultSet, Void> expect(final String... expected) {
-    return new Function1<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          final List<String> lines = new ArrayList<>();
-          BaseGeodeAdapterIT.collect(lines, resultSet);
-          Assert.assertEquals(Arrays.asList(expected), lines);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+    return resultSet -> {
+      try {
+        final List<String> lines = new ArrayList<>();
+        BaseGeodeAdapterIT.collect(lines, resultSet);
+        Assert.assertEquals(Arrays.asList(expected), lines);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
+      return null;
     };
   }
 
@@ -84,15 +82,13 @@ public class BaseGeodeAdapterIT {
   }
 
   protected Function1<ResultSet, Void> output() {
-    return new Function1<ResultSet, Void>() {
-      public Void apply(ResultSet resultSet) {
-        try {
-          output(resultSet, System.out);
-        } catch (SQLException e) {
-          throw new RuntimeException(e);
-        }
-        return null;
+    return resultSet -> {
+      try {
+        output(resultSet, System.out);
+      } catch (SQLException e) {
+        throw new RuntimeException(e);
       }
+      return null;
     };
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
index 82d2c9f..7e8b5dd 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/DefaultQueryable.java
@@ -51,7 +51,7 @@ abstract class DefaultQueryable<T> extends DefaultEnumerable<T>
    * Creates a DefaultQueryable using a factory that records events.
    */
   protected DefaultQueryable() {
-    this(QueryableRecorder.<T>instance());
+    this(QueryableRecorder.instance());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
index 646d95b..9b3dbba 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/EnumerableDefaults.java
@@ -35,7 +35,6 @@ import org.apache.calcite.linq4j.function.Predicate1;
 import org.apache.calcite.linq4j.function.Predicate2;
 
 import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 import java.math.BigDecimal;
@@ -370,7 +369,7 @@ public abstract class EnumerableDefaults {
    * sequence.
    */
   public static <TSource> int count(Enumerable<TSource> enumerable) {
-    return (int) longCount(enumerable, Functions.<TSource>truePredicate1());
+    return (int) longCount(enumerable, Functions.truePredicate1());
   }
 
   /**
@@ -402,36 +401,31 @@ public abstract class EnumerableDefaults {
       TSource value) {
     try (Enumerator<TSource> os = enumerable.enumerator()) {
       if (os.moveNext()) {
-        return Linq4j.asEnumerable(
-            new Iterable<TSource>() {
-              public Iterator<TSource> iterator() {
-                return new Iterator<TSource>() {
+        return Linq4j.asEnumerable(() -> new Iterator<TSource>() {
 
-                  private boolean nonFirst;
+          private boolean nonFirst;
 
-                  private Iterator<TSource> rest;
+          private Iterator<TSource> rest;
 
-                  public boolean hasNext() {
-                    return !nonFirst || rest.hasNext();
-                  }
+          public boolean hasNext() {
+            return !nonFirst || rest.hasNext();
+          }
 
-                  public TSource next() {
-                    if (nonFirst) {
-                      return rest.next();
-                    } else {
-                      final TSource first = os.current();
-                      nonFirst = true;
-                      rest = Linq4j.enumeratorIterator(os);
-                      return first;
-                    }
-                  }
+          public TSource next() {
+            if (nonFirst) {
+              return rest.next();
+            } else {
+              final TSource first = os.current();
+              nonFirst = true;
+              rest = Linq4j.enumeratorIterator(os);
+              return first;
+            }
+          }
 
-                  public void remove() {
-                    throw new UnsupportedOperationException("remove");
-                  }
-                };
-              }
-            });
+          public void remove() {
+            throw new UnsupportedOperationException("remove");
+          }
+        });
       } else {
         return Linq4j.singletonEnumerable(value);
       }
@@ -683,11 +677,7 @@ public abstract class EnumerableDefaults {
       Enumerable<TSource> enumerable, Function1<TSource, TKey> keySelector,
       final Function2<TKey, Enumerable<TSource>, TResult> resultSelector) {
     return enumerable.toLookup(keySelector)
-        .select(new Function1<Grouping<TKey, TSource>, TResult>() {
-          public TResult apply(Grouping<TKey, TSource> group) {
-            return resultSelector.apply(group.getKey(), group);
-          }
-        });
+        .select(group -> resultSelector.apply(group.getKey(), group));
   }
 
   /**
@@ -701,11 +691,7 @@ public abstract class EnumerableDefaults {
       final Function2<TKey, Enumerable<TSource>, TResult> resultSelector,
       EqualityComparer<TKey> comparer) {
     return enumerable.toLookup(keySelector, comparer)
-        .select(new Function1<Grouping<TKey, TSource>, TResult>() {
-          public TResult apply(Grouping<TKey, TSource> group) {
-            return resultSelector.apply(group.getKey(), group);
-          }
-        });
+        .select(group -> resultSelector.apply(group.getKey(), group));
   }
 
   /**
@@ -719,11 +705,7 @@ public abstract class EnumerableDefaults {
       Function1<TSource, TElement> elementSelector,
       final Function2<TKey, Enumerable<TElement>, TResult> resultSelector) {
     return enumerable.toLookup(keySelector, elementSelector)
-        .select(new Function1<Grouping<TKey, TElement>, TResult>() {
-          public TResult apply(Grouping<TKey, TElement> group) {
-            return resultSelector.apply(group.getKey(), group);
-          }
-        });
+        .select(group -> resultSelector.apply(group.getKey(), group));
   }
 
   /**
@@ -739,11 +721,7 @@ public abstract class EnumerableDefaults {
       final Function2<TKey, Enumerable<TElement>, TResult> resultSelector,
       EqualityComparer<TKey> comparer) {
     return enumerable.toLookup(keySelector, elementSelector, comparer)
-        .select(new Function1<Grouping<TKey, TElement>, TResult>() {
-          public TResult apply(Grouping<TKey, TElement> group) {
-            return resultSelector.apply(group.getKey(), group);
-          }
-        });
+        .select(group -> resultSelector.apply(group.getKey(), group));
   }
 
   /**
@@ -907,7 +885,7 @@ public abstract class EnumerableDefaults {
             final Map.Entry<TKey, TSource> entry = entries.current();
             final Enumerable<TInner> inners = innerLookup.get(entry.getKey());
             return resultSelector.apply(entry.getValue(),
-                inners == null ? Linq4j.<TInner>emptyEnumerable() : inners);
+                inners == null ? Linq4j.emptyEnumerable() : inners);
           }
 
           public boolean moveNext() {
@@ -948,7 +926,7 @@ public abstract class EnumerableDefaults {
             final Map.Entry<TKey, TSource> entry = entries.current();
             final Enumerable<TInner> inners = innerLookup.get(entry.getKey());
             return resultSelector.apply(entry.getValue(),
-                inners == null ? Linq4j.<TInner>emptyEnumerable() : inners);
+                inners == null ? Linq4j.emptyEnumerable() : inners);
           }
 
           public boolean moveNext() {
@@ -1327,13 +1305,10 @@ public abstract class EnumerableDefaults {
                 ? inner.select(innerKeySelector).distinct()
                 : inner.select(innerKeySelector).distinct(comparer);
 
-        return EnumerableDefaults.where(outer.enumerator(),
-            new Predicate1<TSource>() {
-              public boolean apply(TSource v0) {
-                final TKey key = outerKeySelector.apply(v0);
-                return innerLookup.contains(key);
-              }
-            });
+        return EnumerableDefaults.where(outer.enumerator(), v0 -> {
+          final TKey key = outerKeySelector.apply(v0);
+          return innerLookup.contains(key);
+        });
       }
     };
   }
@@ -1348,7 +1323,7 @@ public abstract class EnumerableDefaults {
       final boolean generateNullsOnLeft,
       final boolean generateNullsOnRight) {
     // Building the result as a list is easy but hogs memory. We should iterate.
-    final List<TResult> result = Lists.newArrayList();
+    final List<TResult> result = new ArrayList<>();
     final Enumerator<TSource> lefts = outer.enumerator();
     final List<TInner> rightList = inner.toList();
     final Set<TInner> rightUnmatched;
@@ -1516,7 +1491,7 @@ public abstract class EnumerableDefaults {
    * of elements in a sequence.
    */
   public static <TSource> long longCount(Enumerable<TSource> source) {
-    return longCount(source, Functions.<TSource>truePredicate1());
+    return longCount(source, Functions.truePredicate1());
   }
 
   /**
@@ -1675,13 +1650,13 @@ public abstract class EnumerableDefaults {
   @SuppressWarnings("unchecked")
   private static <TSource extends Comparable<TSource>> Function2<TSource, TSource, TSource>
       minFunction() {
-    return (Function2<TSource, TSource, TSource>) Extensions.COMPARABLE_MIN;
+    return (Function2<TSource, TSource, TSource>) (Function2) Extensions.COMPARABLE_MIN;
   }
 
   @SuppressWarnings("unchecked")
   private static <TSource extends Comparable<TSource>> Function2<TSource, TSource, TSource>
       maxFunction() {
-    return (Function2<TSource, TSource, TSource>) Extensions.COMPARABLE_MAX;
+    return (Function2<TSource, TSource, TSource>) (Function2) Extensions.COMPARABLE_MAX;
   }
 
   /**
@@ -1810,7 +1785,7 @@ public abstract class EnumerableDefaults {
       Enumerable<TSource> enumerable, Class<TResult> clazz) {
     //noinspection unchecked
     return (Enumerable) where(enumerable,
-        Functions.<TSource, TResult>ofTypePredicate(clazz));
+        Functions.ofTypePredicate(clazz));
   }
 
   /**
@@ -1834,7 +1809,7 @@ public abstract class EnumerableDefaults {
     // Otherwise there will be a ClassCastException while retrieving.
     final Map<TKey, List<TSource>> map = new TreeMap<>(comparator);
     LookupImpl<TKey, TSource> lookup = toLookup_(map, source, keySelector,
-        Functions.<TSource>identitySelector());
+        Functions.identitySelector());
     return lookup.valuesEnumerable();
   }
 
@@ -1844,7 +1819,7 @@ public abstract class EnumerableDefaults {
    */
   public static <TSource, TKey extends Comparable> Enumerable<TSource> orderByDescending(
       Enumerable<TSource> source, Function1<TSource, TKey> keySelector) {
-    return orderBy(source, keySelector, Collections.<TKey>reverseOrder());
+    return orderBy(source, keySelector, Collections.reverseOrder());
   }
 
   /**
@@ -2307,11 +2282,9 @@ public abstract class EnumerableDefaults {
    */
   public static <TSource> Enumerable<TSource> skip(Enumerable<TSource> source,
       final int count) {
-    return skipWhile(source, new Predicate2<TSource, Integer>() {
-      public boolean apply(TSource v1, Integer v2) {
-        // Count is 1-based
-        return v2 < count;
-      }
+    return skipWhile(source, (v1, v2) -> {
+      // Count is 1-based
+      return v2 < count;
     });
   }
 
@@ -2323,7 +2296,7 @@ public abstract class EnumerableDefaults {
   public static <TSource> Enumerable<TSource> skipWhile(
       Enumerable<TSource> source, Predicate1<TSource> predicate) {
     return skipWhile(source,
-        Functions.<TSource, Integer>toPredicate2(predicate));
+        Functions.toPredicate2(predicate));
   }
 
   /**
@@ -2451,11 +2424,9 @@ public abstract class EnumerableDefaults {
   public static <TSource> Enumerable<TSource> take(Enumerable<TSource> source,
       final int count) {
     return takeWhile(
-        source, new Predicate2<TSource, Integer>() {
-          public boolean apply(TSource v1, Integer v2) {
-            // Count is 1-based
-            return v2 < count;
-          }
+        source, (v1, v2) -> {
+          // Count is 1-based
+          return v2 < count;
         });
   }
 
@@ -2466,11 +2437,9 @@ public abstract class EnumerableDefaults {
   public static <TSource> Enumerable<TSource> take(Enumerable<TSource> source,
       final long count) {
     return takeWhileLong(
-        source, new Predicate2<TSource, Long>() {
-          public boolean apply(TSource v1, Long v2) {
-            // Count is 1-based
-            return v2 < count;
-          }
+        source, (v1, v2) -> {
+          // Count is 1-based
+          return v2 < count;
         });
   }
 
@@ -2481,7 +2450,7 @@ public abstract class EnumerableDefaults {
   public static <TSource> Enumerable<TSource> takeWhile(
       Enumerable<TSource> source, final Predicate1<TSource> predicate) {
     return takeWhile(source,
-        Functions.<TSource, Integer>toPredicate2(predicate));
+        Functions.toPredicate2(predicate));
   }
 
   /**
@@ -2531,7 +2500,7 @@ public abstract class EnumerableDefaults {
   public static <TSource, TKey extends Comparable<TKey>> OrderedEnumerable<TSource> thenBy(
       OrderedEnumerable<TSource> source, Function1<TSource, TKey> keySelector) {
     return createOrderedEnumerable(source, keySelector,
-        Extensions.<TKey>comparableComparator(), false);
+        Extensions.comparableComparator(), false);
   }
 
   /**
@@ -2552,7 +2521,7 @@ public abstract class EnumerableDefaults {
       OrderedEnumerable<TSource> thenByDescending(
       OrderedEnumerable<TSource> source, Function1<TSource, TKey> keySelector) {
     return createOrderedEnumerable(source, keySelector,
-        Extensions.<TKey>comparableComparator(), true);
+        Extensions.comparableComparator(), true);
   }
 
   /**
@@ -2574,7 +2543,7 @@ public abstract class EnumerableDefaults {
    */
   public static <TSource, TKey> Map<TKey, TSource> toMap(
       Enumerable<TSource> source, Function1<TSource, TKey> keySelector) {
-    return toMap(source, keySelector, Functions.<TSource>identitySelector());
+    return toMap(source, keySelector, Functions.identitySelector());
   }
 
   /**
@@ -2585,7 +2554,7 @@ public abstract class EnumerableDefaults {
   public static <TSource, TKey> Map<TKey, TSource> toMap(
       Enumerable<TSource> source, Function1<TSource, TKey> keySelector,
       EqualityComparer<TKey> comparer) {
-    return toMap(source, keySelector, Functions.<TSource>identitySelector(), comparer);
+    return toMap(source, keySelector, Functions.identitySelector(), comparer);
   }
 
   /**
@@ -2656,7 +2625,7 @@ public abstract class EnumerableDefaults {
    */
   public static <TSource, TKey> Lookup<TKey, TSource> toLookup(
       Enumerable<TSource> source, Function1<TSource, TKey> keySelector) {
-    return toLookup(source, keySelector, Functions.<TSource>identitySelector());
+    return toLookup(source, keySelector, Functions.identitySelector());
   }
 
   /**
@@ -2668,7 +2637,7 @@ public abstract class EnumerableDefaults {
       Enumerable<TSource> source, Function1<TSource, TKey> keySelector,
       EqualityComparer<TKey> comparer) {
     return toLookup(
-        source, keySelector, Functions.<TSource>identitySelector(), comparer);
+        source, keySelector, Functions.identitySelector(), comparer);
   }
 
   /**
@@ -2762,20 +2731,12 @@ public abstract class EnumerableDefaults {
   }
 
   private static <TSource> Function1<Wrapped<TSource>, TSource> unwrapper() {
-    return new Function1<Wrapped<TSource>, TSource>() {
-      public TSource apply(Wrapped<TSource> a0) {
-        return a0.element;
-      }
-    };
+    return a0 -> a0.element;
   }
 
   private static <TSource> Function1<TSource, Wrapped<TSource>> wrapperFor(
       final EqualityComparer<TSource> comparer) {
-    return new Function1<TSource, Wrapped<TSource>>() {
-      public Wrapped<TSource> apply(TSource a0) {
-        return Wrapped.upAs(comparer, a0);
-      }
-    };
+    return a0 -> Wrapped.upAs(comparer, a0);
   }
 
   /**
@@ -3324,8 +3285,8 @@ public abstract class EnumerableDefaults {
         rights.add(right);
       }
       cartesians = Linq4j.product(
-          ImmutableList.of(Linq4j.<Object>enumerator(lefts),
-              Linq4j.<Object>enumerator(rights)));
+          ImmutableList.of(Linq4j.enumerator(lefts),
+              Linq4j.enumerator(rights)));
       return true;
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
index c72e88c..7f288da 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Extensions.java
@@ -111,109 +111,51 @@ public abstract class Extensions {
   private Extensions() {}
 
   static final Function2<BigDecimal, BigDecimal, BigDecimal> BIG_DECIMAL_SUM =
-      new Function2<BigDecimal, BigDecimal, BigDecimal>() {
-        public BigDecimal apply(BigDecimal v1, BigDecimal v2) {
-          return v1.add(v2);
-        }
-      };
+      BigDecimal::add;
 
   static final Function2<Float, Float, Float> FLOAT_SUM =
-      new Function2<Float, Float, Float>() {
-        public Float apply(Float v1, Float v2) {
-          return v1 + v2;
-        }
-      };
+      (v1, v2) -> v1 + v2;
 
   static final Function2<Double, Double, Double> DOUBLE_SUM =
-      new Function2<Double, Double, Double>() {
-        public Double apply(Double v1, Double v2) {
-          return v1 + v2;
-        }
-      };
+      (v1, v2) -> v1 + v2;
 
   static final Function2<Integer, Integer, Integer> INTEGER_SUM =
-      new Function2<Integer, Integer, Integer>() {
-        public Integer apply(Integer v1, Integer v2) {
-          return v1 + v2;
-        }
-      };
+      (v1, v2) -> v1 + v2;
 
   static final Function2<Long, Long, Long> LONG_SUM =
-      new Function2<Long, Long, Long>() {
-        public Long apply(Long v1, Long v2) {
-          return v1 + v2;
-        }
-      };
-
-  static final Function2 COMPARABLE_MIN =
-      new Function2<Comparable, Comparable, Comparable>() {
-        public Comparable apply(Comparable v1, Comparable v2) {
-          return v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
-        }
-      };
-
-  static final Function2 COMPARABLE_MAX =
-      new Function2<Comparable, Comparable, Comparable>() {
-        public Comparable apply(Comparable v1, Comparable v2) {
-          return v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 + v2;
+
+  @SuppressWarnings("unchecked")
+  static final Function2<Comparable, Comparable, Comparable> COMPARABLE_MIN =
+      (v1, v2) -> v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
+
+  @SuppressWarnings("unchecked")
+  static final Function2<Comparable, Comparable, Comparable> COMPARABLE_MAX =
+      (v1, v2) -> v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
 
   static final Function2<Float, Float, Float> FLOAT_MIN =
-      new Function2<Float, Float, Float>() {
-        public Float apply(Float v1, Float v2) {
-          return v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
 
   static final Function2<Float, Float, Float> FLOAT_MAX =
-      new Function2<Float, Float, Float>() {
-        public Float apply(Float v1, Float v2) {
-          return v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
 
   static final Function2<Double, Double, Double> DOUBLE_MIN =
-      new Function2<Double, Double, Double>() {
-        public Double apply(Double v1, Double v2) {
-          return v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
 
   static final Function2<Double, Double, Double> DOUBLE_MAX =
-      new Function2<Double, Double, Double>() {
-        public Double apply(Double v1, Double v2) {
-          return v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
 
   static final Function2<Integer, Integer, Integer> INTEGER_MIN =
-      new Function2<Integer, Integer, Integer>() {
-        public Integer apply(Integer v1, Integer v2) {
-          return v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
 
   static final Function2<Integer, Integer, Integer> INTEGER_MAX =
-      new Function2<Integer, Integer, Integer>() {
-        public Integer apply(Integer v1, Integer v2) {
-          return v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
 
   static final Function2<Long, Long, Long> LONG_MIN =
-      new Function2<Long, Long, Long>() {
-        public Long apply(Long v1, Long v2) {
-          return v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) > 0 ? v2 : v1;
 
   static final Function2<Long, Long, Long> LONG_MAX =
-      new Function2<Long, Long, Long>() {
-        public Long apply(Long v1, Long v2) {
-          return v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
-        }
-      };
+      (v1, v2) -> v1 == null || v1.compareTo(v2) < 0 ? v2 : v1;
 
   // flags a piece of code we're yet to implement
   public static RuntimeException todo() {
@@ -228,17 +170,8 @@ public abstract class Extensions {
             Linq4j.DEFAULT_PROVIDER, (Class) Object.class, null, source);
   }
 
-  private static final Comparator<Comparable> COMPARABLE_COMPARATOR =
-      new Comparator<Comparable>() {
-        public int compare(Comparable o1, Comparable o2) {
-          //noinspection unchecked
-          return o1.compareTo(o2);
-        }
-      };
-
   static <T extends Comparable<T>> Comparator<T> comparableComparator() {
-    //noinspection unchecked
-    return (Comparator<T>) (Comparator) COMPARABLE_COMPARATOR;
+    return Comparable::compareTo;
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java b/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
index 65af763..15aaf1f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/GroupingImpl.java
@@ -16,10 +16,9 @@
  */
 package org.apache.calcite.linq4j;
 
-import com.google.common.base.Preconditions;
-
 import java.util.Collection;
 import java.util.Map;
+import java.util.Objects;
 
 /**
  * Implementation of {@link Grouping}.
@@ -33,8 +32,8 @@ class GroupingImpl<K, V> extends AbstractEnumerable<V>
   private final Collection<V> values;
 
   GroupingImpl(K key, Collection<V> values) {
-    this.key = Preconditions.checkNotNull(key);
-    this.values = Preconditions.checkNotNull(values);
+    this.key = Objects.requireNonNull(key);
+    this.values = Objects.requireNonNull(values);
   }
 
   @Override public String toString() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
index 5e58a44..11fc069 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Linq4j.java
@@ -18,8 +18,6 @@ package org.apache.calcite.linq4j;
 
 import org.apache.calcite.linq4j.function.Function1;
 
-import com.google.common.collect.Lists;
-
 import java.lang.reflect.Method;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -390,15 +388,13 @@ public abstract class Linq4j {
   /** Returns the cartesian product of an iterable of iterables. */
   public static <T> Iterable<List<T>> product(
       final Iterable<? extends Iterable<T>> iterables) {
-    return new Iterable<List<T>>() {
-      public Iterator<List<T>> iterator() {
-        final List<Enumerator<T>> enumerators = Lists.newArrayList();
-        for (Iterable<T> iterable : iterables) {
-          enumerators.add(iterableEnumerator(iterable));
-        }
-        return enumeratorIterator(
-            new CartesianProductListEnumerator<>(enumerators));
+    return () -> {
+      final List<Enumerator<T>> enumerators = new ArrayList<>();
+      for (Iterable<T> iterable : iterables) {
+        enumerators.add(iterableEnumerator(iterable));
       }
+      return enumeratorIterator(
+          new CartesianProductListEnumerator<>(enumerators));
     };
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java b/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
index c0e4192..4f5192f 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/Ord.java
@@ -52,11 +52,7 @@ public class Ord<E> implements Map.Entry<Integer, E> {
    * Creates an iterable of {@code Ord}s over an iterable.
    */
   public static <E> Iterable<Ord<E>> zip(final Iterable<? extends E> iterable) {
-    return new Iterable<Ord<E>>() {
-      public Iterator<Ord<E>> iterator() {
-        return zip(iterable.iterator());
-      }
-    };
+    return () -> zip(iterable.iterator());
   }
 
   /**
@@ -114,23 +110,19 @@ public class Ord<E> implements Map.Entry<Integer, E> {
    */
   public static <E> Iterable<Ord<E>> reverse(Iterable<? extends E> elements) {
     final ImmutableList<E> elementList = ImmutableList.copyOf(elements);
-    return new Iterable<Ord<E>>() {
-      public Iterator<Ord<E>> iterator() {
-        return new Iterator<Ord<E>>() {
-          int i = elementList.size() - 1;
-
-          public boolean hasNext() {
-            return i >= 0;
-          }
-
-          public Ord<E> next() {
-            return Ord.of(i, elementList.get(i--));
-          }
-
-          public void remove() {
-            throw new UnsupportedOperationException("remove");
-          }
-        };
+    return () -> new Iterator<Ord<E>>() {
+      int i = elementList.size() - 1;
+
+      public boolean hasNext() {
+        return i >= 0;
+      }
+
+      public Ord<E> next() {
+        return Ord.of(i, elementList.get(i--));
+      }
+
+      public void remove() {
+        throw new UnsupportedOperationException("remove");
       }
     };
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
index 328e9e6..8a21331 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableDefaults.java
@@ -897,7 +897,7 @@ public abstract class QueryableDefaults {
       FunctionExpression<Predicate1<T>> predicate) {
     return skipWhileN(source,
         Expressions.lambda(
-            Functions.<T, Integer>toPredicate2(predicate.getFunction())));
+            Functions.toPredicate2(predicate.getFunction())));
   }
 
   /**
@@ -1033,7 +1033,7 @@ public abstract class QueryableDefaults {
       FunctionExpression<Predicate1<T>> predicate) {
     return takeWhileN(source,
         Expressions.lambda(
-            Functions.<T, Integer>toPredicate2(predicate.getFunction())));
+            Functions.toPredicate2(predicate.getFunction())));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
index 065962b..513a42c 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/QueryableRecorder.java
@@ -71,7 +71,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.aggregate(source, seed, func);
       }
-    }.<TAccumulate>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public <TAccumulate, TResult> TResult aggregate(final Queryable<T> source,
@@ -82,7 +82,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.aggregate(source, seed, func, selector);
       }
-    }.<TResult>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public boolean all(final Queryable<T> source,
@@ -291,7 +291,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.elementAt(source, index);
       }
-    }.<T>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public T elementAtOrDefault(final Queryable<T> source, final int index) {
@@ -299,7 +299,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.elementAtOrDefault(source, index);
       }
-    }.<T>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public Queryable<T> except(final Queryable<T> source,
@@ -572,7 +572,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.max(source);
       }
-    }.<T>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public <TResult extends Comparable<TResult>> TResult max(
@@ -582,7 +582,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.max(source, selector);
       }
-    }.<TResult>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public T min(final Queryable<T> source) {
@@ -590,7 +590,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.min(source);
       }
-    }.<T>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public <TResult extends Comparable<TResult>> TResult min(
@@ -600,7 +600,7 @@ public class QueryableRecorder<T> implements QueryableFactory<T> {
       public void replay(QueryableFactory<T> factory) {
         factory.min(source, selector);
       }
-    }.<TResult>castSingle(); // CHECKSTYLE: IGNORE 0
+    }.castSingle(); // CHECKSTYLE: IGNORE 0
   }
 
   public <TResult> Queryable<TResult> ofType(final Queryable<T> source,

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
index 9181a02..faad8c6 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Function1.java
@@ -28,11 +28,7 @@ public interface Function1<T0, R> extends Function<R> {
    *
    * @see Functions#identitySelector()
    */
-  Function1<Object, Object> IDENTITY = new Function1<Object, Object>() {
-    public Object apply(Object v0) {
-      return v0;
-    }
-  };
+  Function1<Object, Object> IDENTITY = v0 -> v0;
 
   R apply(T0 a0);
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
index f903422..b8f1caa 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Functions.java
@@ -37,7 +37,7 @@ public abstract class Functions {
   private Functions() {}
 
   public static final Map<Class<? extends Function>, Class> FUNCTION_RESULT_TYPES =
-      Collections.<Class<? extends Function>, Class>unmodifiableMap(
+      Collections.unmodifiableMap(
           map(Function0.class, Object.class,
               Function1.class, Object.class,
               Function2.class, Object.class,
@@ -74,19 +74,10 @@ public abstract class Functions {
   private static final EqualityComparer<Object[]> ARRAY_COMPARER =
       new ArrayEqualityComparer();
 
-  private static final Function1 CONSTANT_NULL_FUNCTION1 =
-      new Function1() {
-        public Object apply(Object s) {
-          return null;
-        }
-      };
+  private static final Function1 CONSTANT_NULL_FUNCTION1 = s -> null;
 
   private static final Function1 TO_STRING_FUNCTION1 =
-      new Function1<Object, String>() {
-        public String apply(Object a0) {
-          return a0.toString();
-        }
-      };
+      (Function1<Object, String>) Object::toString;
 
   @SuppressWarnings("unchecked")
   private static <K, V> Map<K, V> map(K k, V v, Object... rest) {
@@ -108,11 +99,7 @@ public abstract class Functions {
 
   /** Returns a 1-parameter function that always returns the same value. */
   public static <T, R> Function1<T, R> constant(final R r) {
-    return new Function1<T, R>() {
-      public R apply(T s) {
-        return r;
-      }
-    };
+    return s -> r;
   }
 
   /** Returns a 1-parameter function that always returns null. */
@@ -194,20 +181,12 @@ public abstract class Functions {
    * @return Predicate that tests for desired type
    */
   public static <T, T2> Predicate1<T> ofTypePredicate(final Class<T2> clazz) {
-    return new Predicate1<T>() {
-      public boolean apply(T v1) {
-        return v1 == null || clazz.isInstance(v1);
-      }
-    };
+    return v1 -> v1 == null || clazz.isInstance(v1);
   }
 
   public static <T1, T2> Predicate2<T1, T2> toPredicate2(
       final Predicate1<T1> p1) {
-    return new Predicate2<T1, T2>() {
-      public boolean apply(T1 v1, T2 v2) {
-        return p1.apply(v1);
-      }
-    };
+    return (v1, v2) -> p1.apply(v1);
   }
 
   /**
@@ -215,11 +194,7 @@ public abstract class Functions {
    */
   public static <T1, T2> Predicate2<T1, T2> toPredicate(
       final Function2<T1, T2, Boolean> function) {
-    return new Predicate2<T1, T2>() {
-      public boolean apply(T1 v1, T2 v2) {
-        return function.apply(v1, v2);
-      }
-    };
+    return function::apply;
   }
 
   /**
@@ -227,11 +202,7 @@ public abstract class Functions {
    */
   private static <T> Predicate1<T> toPredicate(
       final Function1<T, Boolean> function) {
-    return new Predicate1<T>() {
-      public boolean apply(T v1) {
-        return function.apply(v1);
-      }
-    };
+    return function::apply;
   }
 
   /**
@@ -260,11 +231,7 @@ public abstract class Functions {
    */
   public static <T1> Function1<T1, Integer> adapt(
       final IntegerFunction1<T1> f) {
-    return new Function1<T1, Integer>() {
-      public Integer apply(T1 a0) {
-        return f.apply(a0);
-      }
-    };
+    return f::apply;
   }
 
   /**
@@ -272,11 +239,7 @@ public abstract class Functions {
    * an {@link Function1} returning a {@link Double}.
    */
   public static <T1> Function1<T1, Double> adapt(final DoubleFunction1<T1> f) {
-    return new Function1<T1, Double>() {
-      public Double apply(T1 a0) {
-        return f.apply(a0);
-      }
-    };
+    return f::apply;
   }
 
   /**
@@ -284,11 +247,7 @@ public abstract class Functions {
    * an {@link Function1} returning a {@link Long}.
    */
   public static <T1> Function1<T1, Long> adapt(final LongFunction1<T1> f) {
-    return new Function1<T1, Long>() {
-      public Long apply(T1 a0) {
-        return f.apply(a0);
-      }
-    };
+    return f::apply;
   }
 
   /**
@@ -296,11 +255,7 @@ public abstract class Functions {
    * an {@link Function1} returning a {@link Float}.
    */
   public static <T1> Function1<T1, Float> adapt(final FloatFunction1<T1> f) {
-    return new Function1<T1, Float>() {
-      public Float apply(T1 a0) {
-        return f.apply(a0);
-      }
-    };
+    return f::apply;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
index 062a8f0..b8ccaa6 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate1.java
@@ -27,22 +27,14 @@ public interface Predicate1<T0> extends Function<Boolean> {
    *
    * @see Functions#truePredicate1()
    */
-  Predicate1<Object> TRUE = new Predicate1<Object>() {
-    public boolean apply(Object v0) {
-      return true;
-    }
-  };
+  Predicate1<Object> TRUE = v0 -> true;
 
   /**
    * Predicate that always evaluates to {@code false}.
    *
    * @see Functions#falsePredicate1()
    */
-  Predicate1<Object> FALSE = new Predicate1<Object>() {
-    public boolean apply(Object v0) {
-      return false;
-    }
-  };
+  Predicate1<Object> FALSE = v0 -> false;
 
   boolean apply(T0 v0);
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
index c0cbe79..a3f661d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/function/Predicate2.java
@@ -28,22 +28,14 @@ public interface Predicate2<T0, T1> extends Function<Boolean> {
    *
    * @see org.apache.calcite.linq4j.function.Functions#truePredicate1()
    */
-  Predicate2<Object, Object> TRUE = new Predicate2<Object, Object>() {
-    public boolean apply(Object v0, Object v1) {
-      return true;
-    }
-  };
+  Predicate2<Object, Object> TRUE = (v0, v1) -> true;
 
   /**
    * Predicate that always evaluates to {@code false}.
    *
    * @see org.apache.calcite.linq4j.function.Functions#falsePredicate1()
    */
-  Predicate2<Object, Object> FALSE = new Predicate2<Object, Object>() {
-    public boolean apply(Object v0, Object v1) {
-      return false;
-    }
-  };
+  Predicate2<Object, Object> FALSE = (v0, v1) -> false;
 
   boolean apply(T0 v0, T1 v1);
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/d59b639d/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
----------------------------------------------------------------------
diff --git a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
index d0c27e8..227e90d 100644
--- a/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
+++ b/linq4j/src/main/java/org/apache/calcite/linq4j/tree/ClassDeclarationFinder.java
@@ -42,11 +42,7 @@ public class ClassDeclarationFinder extends Shuttle {
 
   private static final Function1<ClassDeclarationFinder,
       ClassDeclarationFinder> DEFAULT_CHILD_FACTORY =
-      new Function1<ClassDeclarationFinder, ClassDeclarationFinder>() {
-      public ClassDeclarationFinder apply(ClassDeclarationFinder a0) {
-        return new DeterministicCodeOptimizer(a0);
-      }
-    };
+      DeterministicCodeOptimizer::new;
 
   /**
    * Creates visitor that uses default optimizer.
@@ -94,20 +90,18 @@ public class ClassDeclarationFinder extends Shuttle {
     try {
       final Constructor<? extends ClassDeclarationFinder> constructor =
           optimizingClass.getConstructor(ClassDeclarationFinder.class);
-      return new Function1<ClassDeclarationFinder, ClassDeclarationFinder>() {
-        public ClassDeclarationFinder apply(ClassDeclarationFinder a0) {
-          try {
-            return constructor.newInstance(a0);
-          } catch (InstantiationException e) {
-            throw new IllegalStateException(
-                "Unable to create optimizer via " + constructor, e);
-          } catch (IllegalAccessException e) {
-            throw new IllegalStateException(
-                "Unable to create optimizer via " + constructor, e);
-          } catch (InvocationTargetException e) {
-            throw new IllegalStateException(
-                "Unable to create optimizer via " + constructor, e);
-          }
+      return a0 -> {
+        try {
+          return constructor.newInstance(a0);
+        } catch (InstantiationException e) {
+          throw new IllegalStateException(
+              "Unable to create optimizer via " + constructor, e);
+        } catch (IllegalAccessException e) {
+          throw new IllegalStateException(
+              "Unable to create optimizer via " + constructor, e);
+        } catch (InvocationTargetException e) {
+          throw new IllegalStateException(
+              "Unable to create optimizer via " + constructor, e);
         }
       };
     } catch (NoSuchMethodException e) {