You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2014/11/14 23:21:43 UTC

incubator-calcite git commit: [CALCITE-465] Remove OneRow and Empty relational expressions; Values will suffice

Repository: incubator-calcite
Updated Branches:
  refs/heads/master 6d79b5ebf -> 71dd68de1


 [CALCITE-465] Remove OneRow and Empty relational expressions; Values will suffice

 Rename class EmptyRuleRules to PruneEmptyRules.


Project: http://git-wip-us.apache.org/repos/asf/incubator-calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-calcite/commit/71dd68de
Tree: http://git-wip-us.apache.org/repos/asf/incubator-calcite/tree/71dd68de
Diff: http://git-wip-us.apache.org/repos/asf/incubator-calcite/diff/71dd68de

Branch: refs/heads/master
Commit: 71dd68de18644754543442d264b698ea4f164fbc
Parents: 6d79b5e
Author: Julian Hyde <ju...@gmail.com>
Authored: Fri Nov 14 12:26:42 2014 -0800
Committer: Julian Hyde <ju...@gmail.com>
Committed: Fri Nov 14 13:27:12 2014 -0800

----------------------------------------------------------------------
 .../adapter/enumerable/EnumerableEmptyRule.java |  48 ----
 .../enumerable/EnumerableOneRowRule.java        |  50 ----
 .../adapter/enumerable/EnumerableRules.java     |   6 -
 .../adapter/enumerable/EnumerableValues.java    |   4 +-
 .../apache/calcite/adapter/jdbc/JdbcRules.java  |   5 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  18 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |   2 -
 .../java/org/apache/calcite/rel/RelInput.java   |   4 +-
 .../java/org/apache/calcite/rel/core/Empty.java |  97 -------
 .../org/apache/calcite/rel/core/OneRow.java     |  71 -----
 .../org/apache/calcite/rel/core/Values.java     |  42 ++-
 .../calcite/rel/externalize/RelJsonReader.java  |  18 +-
 .../calcite/rel/logical/LogicalOneRow.java      |  60 -----
 .../calcite/rel/logical/LogicalValues.java      |  27 +-
 .../calcite/rel/rules/EmptyPruneRules.java      | 257 ------------------
 .../calcite/rel/rules/PruneEmptyRules.java      | 266 +++++++++++++++++++
 .../rel/rules/ReduceExpressionsRule.java        |  24 +-
 .../calcite/rel/rules/ValuesReduceRule.java     |  40 ++-
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |  11 +-
 .../sql2rel/RelStructuredTypeFlattener.java     |   5 -
 .../calcite/sql2rel/SqlToRelConverter.java      |  28 +-
 .../java/org/apache/calcite/tools/Programs.java |   2 -
 .../apache/calcite/test/RelOptRulesTest.java    |  30 +--
 .../org/apache/calcite/test/RelOptRulesTest.xml |  20 +-
 .../calcite/test/SqlToRelConverterTest.xml      |  18 +-
 .../calcite/adapter/spark/SparkRules.java       |   2 +-
 26 files changed, 427 insertions(+), 728 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
deleted file mode 100644
index 8f57d84..0000000
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableEmptyRule.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/*
- * 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.adapter.enumerable;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.core.Empty;
-import org.apache.calcite.rex.RexLiteral;
-
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/** Planner rule that converts an {@link org.apache.calcite.rel.core.Empty}
- * relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
-public class EnumerableEmptyRule extends ConverterRule {
-  EnumerableEmptyRule() {
-    super(Empty.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableEmptyRule");
-  }
-
-  public RelNode convert(RelNode rel) {
-    Empty empty = (Empty) rel;
-    return new EnumerableValues(
-        empty.getCluster(),
-        empty.getRowType(),
-        ImmutableList.<List<RexLiteral>>of(),
-        empty.getTraitSet().replace(EnumerableConvention.INSTANCE));
-  }
-}
-
-// End EnumerableEmptyRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
deleted file mode 100644
index 3602d7f..0000000
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableOneRowRule.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * 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.adapter.enumerable;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.convert.ConverterRule;
-import org.apache.calcite.rel.logical.LogicalOneRow;
-import org.apache.calcite.rex.RexBuilder;
-
-import java.math.BigDecimal;
-import java.util.Collections;
-
-/** Planner rule that converts a
- * {@link org.apache.calcite.rel.logical.LogicalOneRow} relational expression
- * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
-public class EnumerableOneRowRule extends ConverterRule {
-  EnumerableOneRowRule() {
-    super(LogicalOneRow.class, Convention.NONE, EnumerableConvention.INSTANCE,
-        "EnumerableOneRowRule");
-  }
-
-  public RelNode convert(RelNode rel) {
-    LogicalOneRow oneRow = (LogicalOneRow) rel;
-    RexBuilder rexBuilder = rel.getCluster().getRexBuilder();
-    return new EnumerableValues(
-        oneRow.getCluster(),
-        oneRow.getRowType(),
-        Collections.singletonList(
-            Collections.singletonList(
-                rexBuilder.makeExactLiteral(BigDecimal.ZERO))),
-        oneRow.getTraitSet().replace(EnumerableConvention.INSTANCE));
-  }
-}
-
-// End EnumerableOneRowRule.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
index a9c8dd8..62ac2eb 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableRules.java
@@ -72,12 +72,6 @@ public class EnumerableRules {
   public static final EnumerableValuesRule ENUMERABLE_VALUES_RULE =
       new EnumerableValuesRule();
 
-  public static final EnumerableOneRowRule ENUMERABLE_ONE_ROW_RULE =
-      new EnumerableOneRowRule();
-
-  public static final EnumerableEmptyRule ENUMERABLE_EMPTY_RULE =
-      new EnumerableEmptyRule();
-
   public static final EnumerableWindowRule ENUMERABLE_WINDOW_RULE =
       new EnumerableWindowRule();
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
index 36504b1..e4eeeee 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/EnumerableValues.java
@@ -31,6 +31,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Pair;
 
+import com.google.common.collect.ImmutableList;
+
 import java.lang.reflect.Type;
 import java.util.ArrayList;
 import java.util.List;
@@ -39,7 +41,7 @@ import java.util.List;
  * {@link org.apache.calcite.adapter.enumerable.EnumerableConvention enumerable calling convention}. */
 public class EnumerableValues extends Values implements EnumerableRel {
   EnumerableValues(RelOptCluster cluster, RelDataType rowType,
-      List<List<RexLiteral>> tuples, RelTraitSet traitSet) {
+      ImmutableList<ImmutableList<RexLiteral>> tuples, RelTraitSet traitSet) {
     super(cluster, rowType, tuples, traitSet);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
index eb5ff58..cedd3f7 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcRules.java
@@ -904,14 +904,13 @@ public class JdbcRules {
   /** Values operator implemented in JDBC convention. */
   public static class JdbcValues extends Values implements JdbcRel {
     JdbcValues(RelOptCluster cluster, RelDataType rowType,
-        List<List<RexLiteral>> tuples, RelTraitSet traitSet) {
+        ImmutableList<ImmutableList<RexLiteral>> tuples, RelTraitSet traitSet) {
       super(cluster, rowType, tuples, traitSet);
     }
 
     @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
       assert inputs.isEmpty();
-      return new JdbcValues(
-          getCluster(), rowType, tuples, traitSet);
+      return new JdbcValues(getCluster(), rowType, tuples, traitSet);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index f043c66..7d7093b 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -36,11 +36,11 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.rules.AggregateProjectPullUpConstantsRule;
-import org.apache.calcite.rel.rules.EmptyPruneRules;
 import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.MultiJoin;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.rules.ProjectToWindowRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeField;
@@ -1509,14 +1509,14 @@ public abstract class RelOptUtil {
 
   public static void registerAbstractRels(RelOptPlanner planner) {
     planner.addRule(AggregateProjectPullUpConstantsRule.INSTANCE);
-    planner.addRule(EmptyPruneRules.UNION_INSTANCE);
-    planner.addRule(EmptyPruneRules.PROJECT_INSTANCE);
-    planner.addRule(EmptyPruneRules.FILTER_INSTANCE);
-    planner.addRule(EmptyPruneRules.SORT_INSTANCE);
-    planner.addRule(EmptyPruneRules.AGGREGATE_INSTANCE);
-    planner.addRule(EmptyPruneRules.JOIN_LEFT_INSTANCE);
-    planner.addRule(EmptyPruneRules.JOIN_RIGHT_INSTANCE);
-    planner.addRule(EmptyPruneRules.SORT_FETCH_ZERO_INSTANCE);
+    planner.addRule(PruneEmptyRules.UNION_INSTANCE);
+    planner.addRule(PruneEmptyRules.PROJECT_INSTANCE);
+    planner.addRule(PruneEmptyRules.FILTER_INSTANCE);
+    planner.addRule(PruneEmptyRules.SORT_INSTANCE);
+    planner.addRule(PruneEmptyRules.AGGREGATE_INSTANCE);
+    planner.addRule(PruneEmptyRules.JOIN_LEFT_INSTANCE);
+    planner.addRule(PruneEmptyRules.JOIN_RIGHT_INSTANCE);
+    planner.addRule(PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE);
     planner.addRule(ProjectToWindowRule.PROJECT);
     planner.addRule(FilterMergeRule.INSTANCE);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index bdfba29..2f850f5 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -181,8 +181,6 @@ public class CalcitePrepareImpl implements CalcitePrepare {
           EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
           EnumerableRules.ENUMERABLE_VALUES_RULE,
           EnumerableRules.ENUMERABLE_WINDOW_RULE,
-          EnumerableRules.ENUMERABLE_ONE_ROW_RULE,
-          EnumerableRules.ENUMERABLE_EMPTY_RULE,
           EnumerableRules.ENUMERABLE_TABLE_FUNCTION_SCAN_RULE,
           AggregateStarTableRule.INSTANCE,
           AggregateStarTableRule.INSTANCE2,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/RelInput.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelInput.java b/core/src/main/java/org/apache/calcite/rel/RelInput.java
index 38be893..39ff524 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelInput.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelInput.java
@@ -25,6 +25,8 @@ import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ImmutableBitSet;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -82,7 +84,7 @@ public interface RelInput {
 
   RelCollation getCollation();
 
-  List<List<RexLiteral>> getTuples(String tag);
+  ImmutableList<ImmutableList<RexLiteral>> getTuples(String tag);
 
   boolean getBoolean(String tag, boolean default_);
 }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/core/Empty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Empty.java b/core/src/main/java/org/apache/calcite/rel/core/Empty.java
deleted file mode 100644
index 772b946..0000000
--- a/core/src/main/java/org/apache/calcite/rel/core/Empty.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * 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.rel.core;
-
-import org.apache.calcite.plan.Convention;
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.AbstractRelNode;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.RelWriter;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.sql.SqlExplainLevel;
-
-import java.util.List;
-
-/**
- * Relational expression with zero rows.
- *
- * <p>Empty can not be implemented, but serves as a token for rules to match
- * so that empty sections of queries can be eliminated.
- *
- * <p>Rules:
- *
- * <ul>
- * <li>Created by {@code net.sf.farrago.query.FarragoReduceValuesRule}</li>
- * <li>Triggers {@link org.apache.calcite.rel.rules.EmptyPruneRules}</li>
- * </ul>
- *
- * @see org.apache.calcite.rel.logical.LogicalValues
- * @see OneRow
- */
-public class Empty extends AbstractRelNode {
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a new Empty.
-   *
-   * @param cluster Cluster
-   * @param rowType row type for tuples which would be produced by this rel if
-   *                it actually produced any, but it doesn't (see, philosophy is
-   *                good for something after all!)
-   */
-  public Empty(RelOptCluster cluster, RelDataType rowType) {
-    super(cluster, cluster.traitSetOf(Convention.NONE));
-    this.rowType = rowType;
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    assert traitSet.comprises(Convention.NONE);
-    assert inputs.isEmpty();
-    // immutable with no children
-    return this;
-  }
-
-  @Override protected RelDataType deriveRowType() {
-    return rowType;
-  }
-
-  @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return planner.getCostFactory().makeZeroCost();
-  }
-
-  @Override public double getRows() {
-    return 0.0;
-  }
-
-  @Override public RelWriter explainTerms(RelWriter pw) {
-    return super.explainTerms(pw)
-        // For rel digest, include the row type to discriminate
-        // this from other empties with different row types.
-        // For normal EXPLAIN PLAN, omit the type.
-        .itemIf(
-            "type",
-            rowType,
-            pw.getDetailLevel() == SqlExplainLevel.DIGEST_ATTRIBUTES);
-  }
-}
-
-// End Empty.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/OneRow.java b/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
deleted file mode 100644
index a3ccc5b..0000000
--- a/core/src/main/java/org/apache/calcite/rel/core/OneRow.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * 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.rel.core;
-
-import org.apache.calcite.plan.RelOptCluster;
-import org.apache.calcite.plan.RelOptCost;
-import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.rel.AbstractRelNode;
-import org.apache.calcite.rel.RelInput;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-import org.apache.calcite.sql.type.SqlTypeName;
-
-/**
- * Relational expression that always returns one row.
- *
- * <p>It has one column, called "ZERO", containing the value 0.
- *
- * @see Values
- */
-public abstract class OneRow extends AbstractRelNode {
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a <code>OneRow</code>.
-   *
-   * @param cluster   Cluster that this relational expression belongs to
-   * @param traits    Traits
-   */
-  protected OneRow(RelOptCluster cluster, RelTraitSet traits) {
-    super(cluster, traits);
-  }
-
-  /**
-   * Creates a OneRow by parsing serialized output.
-   */
-  protected OneRow(RelInput input) {
-    this(input.getCluster(), input.getTraitSet());
-  }
-
-  //~ Methods ----------------------------------------------------------------
-
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return planner.getCostFactory().makeTinyCost();
-  }
-
-  protected RelDataType deriveRowType() {
-    return deriveOneRowType(getCluster().getTypeFactory());
-  }
-
-  public static RelDataType deriveOneRowType(RelDataTypeFactory typeFactory) {
-    return typeFactory.builder().add("ZERO", SqlTypeName.INTEGER).build();
-  }
-}
-
-// End OneRow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/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 6f1458e..e6a3bfe 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
@@ -21,6 +21,7 @@ import org.apache.calcite.linq4j.function.Functions;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.AbstractRelNode;
 import org.apache.calcite.rel.RelInput;
@@ -33,6 +34,9 @@ import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Pair;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableList;
+
 import java.util.List;
 
 /**
@@ -43,9 +47,9 @@ public abstract class Values extends AbstractRelNode {
   /**
    * Lambda that helps render tuples as strings.
    */
-  private static final Function1<List<RexLiteral>, Object> F =
-      new Function1<List<RexLiteral>, Object>() {
-        public Object apply(List<RexLiteral> tuple) {
+  private static final Function1<ImmutableList<RexLiteral>, Object> F =
+      new Function1<ImmutableList<RexLiteral>, Object>() {
+        public Object apply(ImmutableList<RexLiteral> tuple) {
           String s = tuple.toString();
           assert s.startsWith("[");
           assert s.endsWith("]");
@@ -53,9 +57,23 @@ public abstract class Values extends AbstractRelNode {
         }
       };
 
+  /** 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 Predicate<Values>() {
+        public boolean apply(Values values) {
+          return values.getTuples().isEmpty();
+        }
+      };
+
   //~ Instance fields --------------------------------------------------------
 
-  protected final List<List<RexLiteral>> tuples;
+  protected final ImmutableList<ImmutableList<RexLiteral>> tuples;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -75,7 +93,7 @@ public abstract class Values extends AbstractRelNode {
   protected Values(
       RelOptCluster cluster,
       RelDataType rowType,
-      List<List<RexLiteral>> tuples,
+      ImmutableList<ImmutableList<RexLiteral>> tuples,
       RelTraitSet traits) {
     super(cluster, traits);
     this.rowType = rowType;
@@ -93,20 +111,18 @@ public abstract class Values extends AbstractRelNode {
 
   //~ Methods ----------------------------------------------------------------
 
-  public List<List<RexLiteral>> getTuples(RelInput input) {
+  public ImmutableList<ImmutableList<RexLiteral>> getTuples(RelInput input) {
     return input.getTuples("tuples");
   }
 
-  /**
-   * @return rows of literals represented by this rel
-   */
-  public List<List<RexLiteral>> getTuples() {
+  /** Returns the rows of literals represented by this Values relational
+   * expression. */
+  public ImmutableList<ImmutableList<RexLiteral>> getTuples() {
     return tuples;
   }
 
-  /**
-   * @return true if all tuples match rowType; otherwise, assert on mismatch
-   */
+  /** Returns true if all tuples match rowType; otherwise, assert on
+   * mismatch. */
   private boolean assertRowType() {
     for (List<RexLiteral> tuple : tuples) {
       assert tuple.size() == rowType.getFieldCount();

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
index 32ea764..c417318 100644
--- a/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
+++ b/core/src/main/java/org/apache/calcite/rel/externalize/RelJsonReader.java
@@ -220,21 +220,23 @@ public class RelJsonReader {
         return relJson.toCollation((List) get("collation"));
       }
 
-      public List<List<RexLiteral>> getTuples(String tag) {
+      public ImmutableList<ImmutableList<RexLiteral>> getTuples(String tag) {
         List<List> jsonTuples = (List) get(tag);
-        final List<List<RexLiteral>> list = new ArrayList<List<RexLiteral>>();
+        final ImmutableList.Builder<ImmutableList<RexLiteral>> builder =
+            ImmutableList.builder();
         for (List jsonTuple : jsonTuples) {
-          list.add(getTuple(jsonTuple));
+          builder.add(getTuple(jsonTuple));
         }
-        return list;
+        return builder.build();
       }
 
-      public List<RexLiteral> getTuple(List jsonTuple) {
-        final List<RexLiteral> list = new ArrayList<RexLiteral>();
+      public ImmutableList<RexLiteral> getTuple(List jsonTuple) {
+        final ImmutableList.Builder<RexLiteral> builder =
+            ImmutableList.builder();
         for (Object jsonValue : jsonTuple) {
-          list.add((RexLiteral) relJson.toRex(this, jsonValue));
+          builder.add((RexLiteral) relJson.toRex(this, jsonValue));
         }
-        return list;
+        return builder.build();
       }
     };
     try {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
deleted file mode 100644
index 4a44459..0000000
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalOneRow.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * 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.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.RelInput;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.OneRow;
-
-import java.util.List;
-
-/**
- * Sub-class of {@link org.apache.calcite.rel.core.OneRow}
- * not targeted at any particular engine or calling convention.
- */
-public final class LogicalOneRow extends OneRow {
-  //~ Constructors -----------------------------------------------------------
-
-  /**
-   * Creates a <code>LogicalOneRow</code>.
-   *
-   * @param cluster   Cluster that this relational expression belongs to
-   */
-  public LogicalOneRow(RelOptCluster cluster) {
-    super(
-        cluster,
-        cluster.traitSetOf(Convention.NONE));
-  }
-
-  /**
-   * Creates a LogicalOneRow by parsing serialized output.
-   */
-  public LogicalOneRow(RelInput input) {
-    super(input.getCluster(), input.getTraitSet());
-  }
-
-  @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    assert traitSet.containsIfApplicable(Convention.NONE);
-    assert inputs.isEmpty();
-    return this;
-  }
-}
-
-// End LogicalOneRow.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/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 fc3beed..5d80851 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
@@ -25,7 +25,11 @@ import org.apache.calcite.rel.RelShuttle;
 import org.apache.calcite.rel.core.Values;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.sql.type.SqlTypeName;
 
+import com.google.common.collect.ImmutableList;
+
+import java.math.BigDecimal;
 import java.util.List;
 
 /**
@@ -51,7 +55,7 @@ public class LogicalValues extends Values {
   public LogicalValues(
       RelOptCluster cluster,
       RelDataType rowType,
-      List<List<RexLiteral>> tuples) {
+      ImmutableList<ImmutableList<RexLiteral>> tuples) {
     super(cluster, rowType, tuples, cluster.traitSetOf(Convention.NONE));
   }
 
@@ -71,6 +75,27 @@ public class LogicalValues extends Values {
         tuples);
   }
 
+  /** Creates a LogicalValues that outputs no rows of a given row type. */
+  public static LogicalValues createEmpty(RelOptCluster cluster,
+      RelDataType rowType) {
+    return new LogicalValues(cluster, rowType,
+        ImmutableList.<ImmutableList<RexLiteral>>of());
+  }
+
+  /** Creates a LogicalValues that outputs one row and one column. */
+  public static LogicalValues createOneRow(RelOptCluster cluster) {
+    final RelDataType rowType =
+        cluster.getTypeFactory().builder()
+            .add("ZERO", SqlTypeName.INTEGER).nullable(false)
+            .build();
+    final ImmutableList<ImmutableList<RexLiteral>> tuples =
+        ImmutableList.of(
+            ImmutableList.of(
+                cluster.getRexBuilder().makeExactLiteral(BigDecimal.ZERO,
+                    rowType.getFieldList().get(0).getType())));
+    return new LogicalValues(cluster, rowType, tuples);
+  }
+
   @Override public RelNode accept(RelShuttle shuttle) {
     return shuttle.visit(this);
   }

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java b/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
deleted file mode 100644
index 9f3cd67..0000000
--- a/core/src/main/java/org/apache/calcite/rel/rules/EmptyPruneRules.java
+++ /dev/null
@@ -1,257 +0,0 @@
-/*
- * 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.rel.rules;
-
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptUtil;
-import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.SingleRel;
-import org.apache.calcite.rel.core.Aggregate;
-import org.apache.calcite.rel.core.Empty;
-import org.apache.calcite.rel.core.Filter;
-import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.Project;
-import org.apache.calcite.rel.core.Sort;
-import org.apache.calcite.rel.logical.LogicalUnion;
-import org.apache.calcite.rex.RexLiteral;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.apache.calcite.plan.RelOptRule.any;
-import static org.apache.calcite.plan.RelOptRule.none;
-import static org.apache.calcite.plan.RelOptRule.operand;
-import static org.apache.calcite.plan.RelOptRule.some;
-import static org.apache.calcite.plan.RelOptRule.unordered;
-
-/**
- * Collection of rules which remove sections of a query plan known never to
- * produce any rows.
- *
- * @see org.apache.calcite.rel.core.Empty
- */
-public abstract class EmptyPruneRules {
-  //~ Static fields/initializers ---------------------------------------------
-
-  /**
-   * Rule that removes empty children of a
-   * {@link org.apache.calcite.rel.logical.LogicalUnion}.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Union(Rel, Empty, Rel2) becomes Union(Rel, Rel2)
-   * <li>Union(Rel, Empty, Empty) becomes Rel
-   * <li>Union(Empty, Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule UNION_INSTANCE =
-      new RelOptRule(
-          operand(LogicalUnion.class,
-              unordered(operand(Empty.class, none()))),
-          "Union") {
-        public void onMatch(RelOptRuleCall call) {
-          LogicalUnion union = call.rel(0);
-          final List<RelNode> childRels = call.getChildRels(union);
-          final List<RelNode> newChildRels = new ArrayList<RelNode>();
-          for (RelNode childRel : childRels) {
-            if (!(childRel instanceof Empty)) {
-              newChildRels.add(childRel);
-            }
-          }
-          assert newChildRels.size() < childRels.size()
-              : "planner promised us at least one Empty child";
-          RelNode newRel;
-          switch (newChildRels.size()) {
-          case 0:
-            newRel = empty(union);
-            break;
-          case 1:
-            newRel =
-                RelOptUtil.createCastRel(
-                    newChildRels.get(0),
-                    union.getRowType(),
-                    true);
-            break;
-          default:
-            newRel =
-                new LogicalUnion(
-                    union.getCluster(),
-                    newChildRels,
-                    union.all);
-            break;
-          }
-          call.transformTo(newRel);
-        }
-      };
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalProject}
-   * to empty if its child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Project(Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule PROJECT_INSTANCE =
-      new RemoveEmptySingleRule(Project.class, "PruneEmptyProject");
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter}
-   * to empty if its child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Filter(Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule FILTER_INSTANCE =
-      new RemoveEmptySingleRule(Filter.class, "PruneEmptyFilter");
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
-   * to empty if its child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Sort(Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule SORT_INSTANCE =
-      new RemoveEmptySingleRule(Sort.class, "PruneEmptySort");
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
-   * to empty if it has {@code LIMIT 0}.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Sort(Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule SORT_FETCH_ZERO_INSTANCE =
-      new RelOptRule(
-          operand(Sort.class, any()), "PruneSortLimit0") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          Sort sort = call.rel(0);
-          if (sort.fetch != null
-              && RexLiteral.intValue(sort.fetch) == 0) {
-            call.transformTo(empty(sort));
-          }
-        }
-      };
-
-  /**
-   * Rule that converts an {@link org.apache.calcite.rel.core.Aggregate}
-   * to empty if its child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Aggregate(Empty) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule AGGREGATE_INSTANCE =
-      new RemoveEmptySingleRule(Aggregate.class, "PruneEmptyAggregate");
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
-   * to empty if its left child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Join(Empty, Scan(Dept), INNER) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule JOIN_LEFT_INSTANCE =
-      new RelOptRule(
-          operand(Join.class,
-              some(
-                  operand(Empty.class, none()),
-                  operand(RelNode.class, any()))),
-              "PruneEmptyJoin(left)") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          Join join = call.rel(0);
-          if (join.getJoinType().generatesNullsOnLeft()) {
-            // "select * from emp right join dept" is not necessarily empty if
-            // emp is empty
-            return;
-          }
-          call.transformTo(empty(join));
-        }
-      };
-
-  /**
-   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
-   * to empty if its right child is empty.
-   *
-   * <p>Examples:
-   *
-   * <ul>
-   * <li>Join(Scan(Emp), Empty, INNER) becomes Empty
-   * </ul>
-   */
-  public static final RelOptRule JOIN_RIGHT_INSTANCE =
-      new RelOptRule(
-          operand(Join.class,
-              some(
-                  operand(RelNode.class, any()),
-                  operand(Empty.class, none()))),
-              "PruneEmptyJoin(right)") {
-        @Override public void onMatch(RelOptRuleCall call) {
-          Join join = call.rel(0);
-          if (join.getJoinType().generatesNullsOnRight()) {
-            // "select * from emp left join dept" is not necessarily empty if
-            // dept is empty
-            return;
-          }
-          call.transformTo(empty(join));
-        }
-      };
-
-  /** Creates an {@link org.apache.calcite.rel.core.Empty} to replace
-   * {@code node}. */
-  private static Empty empty(RelNode node) {
-    return new Empty(node.getCluster(), node.getRowType());
-  }
-
-  /** Planner rule that converts a single-rel (e.g. project, sort, aggregate or
-   * filter) on top of the empty relational expression into empty. */
-  private static class RemoveEmptySingleRule extends RelOptRule {
-    public RemoveEmptySingleRule(Class<? extends SingleRel> clazz,
-        String description) {
-      super(
-          operand(clazz,
-              operand(Empty.class, none())),
-          description);
-    }
-
-    public void onMatch(RelOptRuleCall call) {
-      SingleRel single = call.rel(0);
-      call.transformTo(empty(single));
-    }
-  }
-}
-
-// End EmptyPruneRules.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/rules/PruneEmptyRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/PruneEmptyRules.java b/core/src/main/java/org/apache/calcite/rel/rules/PruneEmptyRules.java
new file mode 100644
index 0000000..008f9aa
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/PruneEmptyRules.java
@@ -0,0 +1,266 @@
+/*
+ * 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.rel.rules;
+
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptUtil;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.calcite.rel.core.Aggregate;
+import org.apache.calcite.rel.core.Filter;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.Project;
+import org.apache.calcite.rel.core.Sort;
+import org.apache.calcite.rel.core.Values;
+import org.apache.calcite.rel.logical.LogicalUnion;
+import org.apache.calcite.rel.logical.LogicalValues;
+import org.apache.calcite.rex.RexLiteral;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.calcite.plan.RelOptRule.any;
+import static org.apache.calcite.plan.RelOptRule.none;
+import static org.apache.calcite.plan.RelOptRule.operand;
+import static org.apache.calcite.plan.RelOptRule.some;
+import static org.apache.calcite.plan.RelOptRule.unordered;
+
+/**
+ * Collection of rules which remove sections of a query plan known never to
+ * produce any rows.
+ *
+ * <p>Conventionally, the way to represent an empty relational expression is
+ * with a {@link Values} that has no tuples.
+ *
+ * @see LogicalValues#createEmpty
+ */
+public abstract class PruneEmptyRules {
+  //~ Static fields/initializers ---------------------------------------------
+
+  /**
+   * Rule that removes empty children of a
+   * {@link org.apache.calcite.rel.logical.LogicalUnion}.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Union(Rel, Empty, Rel2) becomes Union(Rel, Rel2)
+   * <li>Union(Rel, Empty, Empty) becomes Rel
+   * <li>Union(Empty, Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule UNION_INSTANCE =
+      new RelOptRule(
+          operand(LogicalUnion.class,
+              unordered(operand(Values.class, null, Values.IS_EMPTY, none()))),
+          "Union") {
+        public void onMatch(RelOptRuleCall call) {
+          LogicalUnion union = call.rel(0);
+          final List<RelNode> childRels = call.getChildRels(union);
+          final List<RelNode> newChildRels = new ArrayList<RelNode>();
+          for (RelNode childRel : childRels) {
+            if (!isEmpty(childRel)) {
+              newChildRels.add(childRel);
+            }
+          }
+          assert newChildRels.size() < childRels.size()
+              : "planner promised us at least one Empty child";
+          RelNode newRel;
+          switch (newChildRels.size()) {
+          case 0:
+            newRel = empty(union);
+            break;
+          case 1:
+            newRel =
+                RelOptUtil.createCastRel(
+                    newChildRels.get(0),
+                    union.getRowType(),
+                    true);
+            break;
+          default:
+            newRel =
+                new LogicalUnion(
+                    union.getCluster(),
+                    newChildRels,
+                    union.all);
+            break;
+          }
+          call.transformTo(newRel);
+        }
+      };
+
+  private static boolean isEmpty(RelNode node) {
+    return node instanceof Values
+        && ((Values) node).getTuples().isEmpty();
+  }
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalProject}
+   * to empty if its child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Project(Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule PROJECT_INSTANCE =
+      new RemoveEmptySingleRule(Project.class, "PruneEmptyProject");
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.logical.LogicalFilter}
+   * to empty if its child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Filter(Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule FILTER_INSTANCE =
+      new RemoveEmptySingleRule(Filter.class, "PruneEmptyFilter");
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
+   * to empty if its child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Sort(Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule SORT_INSTANCE =
+      new RemoveEmptySingleRule(Sort.class, "PruneEmptySort");
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.core.Sort}
+   * to empty if it has {@code LIMIT 0}.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Sort(Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule SORT_FETCH_ZERO_INSTANCE =
+      new RelOptRule(
+          operand(Sort.class, any()), "PruneSortLimit0") {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Sort sort = call.rel(0);
+          if (sort.fetch != null
+              && RexLiteral.intValue(sort.fetch) == 0) {
+            call.transformTo(empty(sort));
+          }
+        }
+      };
+
+  /**
+   * Rule that converts an {@link org.apache.calcite.rel.core.Aggregate}
+   * to empty if its child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Aggregate(Empty) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule AGGREGATE_INSTANCE =
+      new RemoveEmptySingleRule(Aggregate.class, "PruneEmptyAggregate");
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
+   * to empty if its left child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Join(Empty, Scan(Dept), INNER) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule JOIN_LEFT_INSTANCE =
+      new RelOptRule(
+          operand(Join.class,
+              some(
+                  operand(Values.class, null, Values.IS_EMPTY, none()),
+                  operand(RelNode.class, any()))),
+              "PruneEmptyJoin(left)") {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Join join = call.rel(0);
+          if (join.getJoinType().generatesNullsOnLeft()) {
+            // "select * from emp right join dept" is not necessarily empty if
+            // emp is empty
+            return;
+          }
+          call.transformTo(empty(join));
+        }
+      };
+
+  /**
+   * Rule that converts a {@link org.apache.calcite.rel.core.Join}
+   * to empty if its right child is empty.
+   *
+   * <p>Examples:
+   *
+   * <ul>
+   * <li>Join(Scan(Emp), Empty, INNER) becomes Empty
+   * </ul>
+   */
+  public static final RelOptRule JOIN_RIGHT_INSTANCE =
+      new RelOptRule(
+          operand(Join.class,
+              some(
+                  operand(RelNode.class, any()),
+                  operand(Values.class, null, Values.IS_EMPTY, none()))),
+              "PruneEmptyJoin(right)") {
+        @Override public void onMatch(RelOptRuleCall call) {
+          Join join = call.rel(0);
+          if (join.getJoinType().generatesNullsOnRight()) {
+            // "select * from emp left join dept" is not necessarily empty if
+            // dept is empty
+            return;
+          }
+          call.transformTo(empty(join));
+        }
+      };
+
+  /** Creates a {@link org.apache.calcite.rel.core.Values} to replace
+   * {@code node}. */
+  private static Values empty(RelNode node) {
+    return LogicalValues.createEmpty(node.getCluster(), node.getRowType());
+  }
+
+  /** Planner rule that converts a single-rel (e.g. project, sort, aggregate or
+   * filter) on top of the empty relational expression into empty. */
+  private static class RemoveEmptySingleRule extends RelOptRule {
+    public RemoveEmptySingleRule(Class<? extends SingleRel> clazz,
+        String description) {
+      super(
+          operand(clazz,
+              operand(Values.class, null, Values.IS_EMPTY, none())),
+          description);
+    }
+
+    public void onMatch(RelOptRuleCall call) {
+      SingleRel single = call.rel(0);
+      call.transformTo(empty(single));
+    }
+  }
+}
+
+// End PruneEmptyRules.java

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
index 5fb7035..620096e 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceExpressionsRule.java
@@ -21,11 +21,11 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Empty;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.logical.LogicalCalc;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexBuilder;
@@ -80,8 +80,8 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
   /**
    * Singleton rule that reduces constants inside a
    * {@link org.apache.calcite.rel.logical.LogicalFilter}. If the condition is a
-   * constant, the filter is removed (if TRUE) or replaced with
-   * {@link org.apache.calcite.rel.core.Empty} (if FALSE or NULL).
+   * constant, the filter is removed (if TRUE) or replaced with an empty
+   * {@link org.apache.calcite.rel.core.Values} (if FALSE or NULL).
    */
   public static final ReduceExpressionsRule FILTER_INSTANCE =
       new ReduceExpressionsRule(LogicalFilter.class,
@@ -106,18 +106,14 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
           if (newConditionExp.isAlwaysTrue()) {
             call.transformTo(
                 filter.getInput());
-          } else if (
-              (newConditionExp instanceof RexLiteral)
-                  || RexUtil.isNullLiteral(newConditionExp, true)) {
+          } else if (newConditionExp instanceof RexLiteral
+              || RexUtil.isNullLiteral(newConditionExp, true)) {
             call.transformTo(
-                new Empty(
-                    filter.getCluster(),
+                LogicalValues.createEmpty(filter.getCluster(),
                     filter.getRowType()));
           } else if (reduced) {
             call.transformTo(
-                RelOptUtil.createFilter(
-                    filter.getInput(),
-                    expList.get(0)));
+                RelOptUtil.createFilter(filter.getInput(), expList.get(0)));
           } else {
             if (newConditionExp instanceof RexCall) {
               RexCall rexCall = (RexCall) newConditionExp;
@@ -168,8 +164,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
                 call.transformTo(filter.getInput());
               } else {
                 call.transformTo(
-                    new Empty(
-                        filter.getCluster(),
+                    LogicalValues.createEmpty(filter.getCluster(),
                         filter.getRowType()));
               }
             }
@@ -263,8 +258,7 @@ public abstract class ReduceExpressionsRule extends RelOptRule {
                 // condition is always NULL or FALSE - replace calc
                 // with empty
                 call.transformTo(
-                    new Empty(
-                        calc.getCluster(),
+                    LogicalValues.createEmpty(calc.getCluster(),
                         calc.getRowType()));
                 return;
               } else {

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
index d3f8d6f..533c027 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ValuesReduceRule.java
@@ -20,7 +20,6 @@ import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.rel.core.Empty;
 import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalValues;
@@ -34,6 +33,8 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
+import com.google.common.collect.ImmutableList;
+
 import java.util.ArrayList;
 import java.util.List;
 import java.util.logging.Logger;
@@ -42,9 +43,8 @@ import java.util.logging.Logger;
  * Planner rule that folds projections and filters into an underlying
  * {@link org.apache.calcite.rel.logical.LogicalValues}.
  *
- * <p>Returns a simplified {@code Values},
- * or an {@link org.apache.calcite.rel.core.Empty} if all rows are
- * filtered away.
+ * <p>Returns a simplified {@code Values}, perhaps containing zero tuples
+ * if all rows are filtered away.
  *
  * <p>For example,</p>
  *
@@ -175,8 +175,8 @@ public abstract class ValuesReduceRule extends RelOptRule {
     ReduceExpressionsRule.reduceExpressions(values, reducibleExps);
 
     int changeCount = 0;
-    final List<List<RexLiteral>> tupleList =
-        new ArrayList<List<RexLiteral>>();
+    final ImmutableList.Builder<ImmutableList<RexLiteral>> tuplesBuilder =
+        ImmutableList.builder();
     for (int row = 0; row < values.getTuples().size(); ++row) {
       int i = 0;
       RexNode reducedValue;
@@ -189,23 +189,26 @@ public abstract class ValuesReduceRule extends RelOptRule {
         }
       }
 
-      List<RexLiteral> valuesList = new ArrayList<RexLiteral>();
+      ImmutableList<RexLiteral> valuesList;
       if (projectExprs != null) {
         ++changeCount;
+        final ImmutableList.Builder<RexLiteral> tupleBuilder =
+            ImmutableList.builder();
         for (; i < fieldsPerRow; ++i) {
           reducedValue = reducibleExps.get((row * fieldsPerRow) + i);
           if (reducedValue instanceof RexLiteral) {
-            valuesList.add((RexLiteral) reducedValue);
+            tupleBuilder.add((RexLiteral) reducedValue);
           } else if (RexUtil.isNullLiteral(reducedValue, true)) {
-            valuesList.add(rexBuilder.constantNull());
+            tupleBuilder.add(rexBuilder.constantNull());
           } else {
             return;
           }
         }
+        valuesList = tupleBuilder.build();
       } else {
         valuesList = values.getTuples().get(row);
       }
-      tupleList.add(valuesList);
+      tuplesBuilder.add(valuesList);
     }
 
     if (changeCount > 0) {
@@ -215,21 +218,10 @@ public abstract class ValuesReduceRule extends RelOptRule {
       } else {
         rowType = values.getRowType();
       }
-      final RelNode newRel;
-      if (tupleList.isEmpty()) {
-        newRel =
-            new Empty(
-                values.getCluster(),
-                rowType);
-      } else {
-        newRel =
-            new LogicalValues(
-                values.getCluster(),
-                rowType,
-                tupleList);
-      }
+      final RelNode newRel =
+          new LogicalValues(values.getCluster(), rowType,
+              tuplesBuilder.build());
       call.transformTo(newRel);
-
     } else {
       // Filter had no effect, so we can say that Filter(Values) ==
       // Values.

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
index 8885444..a828870 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -957,13 +957,14 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
       return new TrimResult(values, mapping);
     }
 
-    List<List<RexLiteral>> newTuples = new ArrayList<List<RexLiteral>>();
-    for (List<RexLiteral> tuple : values.getTuples()) {
-      List<RexLiteral> newTuple = new ArrayList<RexLiteral>();
+    final ImmutableList.Builder<ImmutableList<RexLiteral>> newTuples =
+        ImmutableList.builder();
+    for (ImmutableList<RexLiteral> tuple : values.getTuples()) {
+      ImmutableList.Builder<RexLiteral> newTuple = ImmutableList.builder();
       for (int field : fieldsUsed) {
         newTuple.add(tuple.get(field));
       }
-      newTuples.add(newTuple);
+      newTuples.add(newTuple.build());
     }
 
     final Mapping mapping = createMapping(fieldsUsed, fieldCount);
@@ -971,7 +972,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
         RelOptUtil.permute(values.getCluster().getTypeFactory(), rowType,
             mapping);
     final LogicalValues newValues =
-        new LogicalValues(values.getCluster(), newRowType, newTuples);
+        new LogicalValues(values.getCluster(), newRowType, newTuples.build());
     return new TrimResult(newValues, mapping);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
index d30531e..f3557c6 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelStructuredTypeFlattener.java
@@ -36,7 +36,6 @@ import org.apache.calcite.rel.logical.LogicalFilter;
 import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalMinus;
-import org.apache.calcite.rel.logical.LogicalOneRow;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
 import org.apache.calcite.rel.logical.LogicalTableModify;
@@ -464,10 +463,6 @@ public class RelStructuredTypeFlattener implements ReflectiveVisitor {
     rewriteGeneric(rel);
   }
 
-  public void rewriteRel(LogicalOneRow rel) {
-    rewriteGeneric(rel);
-  }
-
   public void rewriteRel(LogicalValues rel) {
     // NOTE jvs 30-Apr-2006:  UDT instances require invocation
     // of a constructor method, which can't be represented

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
index c7a3d0a..ee886f2 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/SqlToRelConverter.java
@@ -46,7 +46,6 @@ import org.apache.calcite.rel.logical.LogicalAggregate;
 import org.apache.calcite.rel.logical.LogicalIntersect;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.rel.logical.LogicalMinus;
-import org.apache.calcite.rel.logical.LogicalOneRow;
 import org.apache.calcite.rel.logical.LogicalProject;
 import org.apache.calcite.rel.logical.LogicalTableFunctionScan;
 import org.apache.calcite.rel.logical.LogicalTableModify;
@@ -1426,8 +1425,8 @@ public class SqlToRelConverter {
     // subqueries), we union each row in as a projection on top of a
     // LogicalOneRow.
 
-    final List<List<RexLiteral>> tupleList =
-        new ArrayList<List<RexLiteral>>();
+    final ImmutableList.Builder<ImmutableList<RexLiteral>> tupleList =
+        ImmutableList.builder();
     final RelDataType rowType;
     if (targetRowType != null) {
       rowType = targetRowType;
@@ -1444,14 +1443,14 @@ public class SqlToRelConverter {
       SqlBasicCall call;
       if (isRowConstructor(node)) {
         call = (SqlBasicCall) node;
-        List<RexLiteral> tuple = new ArrayList<RexLiteral>();
-        for (SqlNode operand : call.operands) {
+        ImmutableList.Builder<RexLiteral> tuple = ImmutableList.builder();
+        for (Ord<SqlNode> operand : Ord.zip(call.operands)) {
           RexLiteral rexLiteral =
               convertLiteralInValuesList(
-                  operand,
+                  operand.e,
                   bb,
                   rowType,
-                  tuple.size());
+                  operand.i);
           if ((rexLiteral == null) && allowLiteralsOnly) {
             return null;
           }
@@ -1463,7 +1462,7 @@ public class SqlToRelConverter {
           tuple.add(rexLiteral);
         }
         if (tuple != null) {
-          tupleList.add(tuple);
+          tupleList.add(tuple.build());
           continue;
         }
       } else {
@@ -1474,8 +1473,7 @@ public class SqlToRelConverter {
                 rowType,
                 0);
         if ((rexLiteral != null) && shouldCreateValuesRel) {
-          tupleList.add(
-              Collections.singletonList(rexLiteral));
+          tupleList.add(ImmutableList.of(rexLiteral));
           continue;
         } else {
           if ((rexLiteral == null) && allowLiteralsOnly) {
@@ -1495,12 +1493,12 @@ public class SqlToRelConverter {
         new LogicalValues(
             cluster,
             rowType,
-            tupleList);
+            tupleList.build());
     RelNode resultRel;
     if (unionInputs.isEmpty()) {
       resultRel = values;
     } else {
-      if (!tupleList.isEmpty()) {
+      if (!values.getTuples().isEmpty()) {
         unionInputs.add(values);
       }
       LogicalUnion union =
@@ -1940,7 +1938,7 @@ public class SqlToRelConverter {
       replaceSubqueries(bb, node, RelOptUtil.Logic.TRUE_FALSE_UNKNOWN);
       final RelNode childRel =
           RelOptUtil.createProject(
-              (null != bb.root) ? bb.root : new LogicalOneRow(cluster),
+              (null != bb.root) ? bb.root : LogicalValues.createOneRow(cluster),
               Collections.singletonList(bb.convertExpression(node)),
               Collections.singletonList(validator.deriveAlias(node, 0)),
               true);
@@ -3602,7 +3600,7 @@ public class SqlToRelConverter {
 
         RelNode projRel =
             RelOptUtil.createProject(
-                new LogicalOneRow(cluster),
+                LogicalValues.createOneRow(cluster),
                 selectList,
                 fieldNameList);
 
@@ -3810,7 +3808,7 @@ public class SqlToRelConverter {
       }
       RelNode in =
           (null == tmpBb.root)
-              ? new LogicalOneRow(cluster)
+              ? LogicalValues.createOneRow(cluster)
               : tmpBb.root;
       unionRels.add(
           RelOptUtil.createProject(

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/main/java/org/apache/calcite/tools/Programs.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/Programs.java b/core/src/main/java/org/apache/calcite/tools/Programs.java
index 49c77ae..e9aa502 100644
--- a/core/src/main/java/org/apache/calcite/tools/Programs.java
+++ b/core/src/main/java/org/apache/calcite/tools/Programs.java
@@ -111,8 +111,6 @@ public class Programs {
           EnumerableRules.ENUMERABLE_TABLE_MODIFICATION_RULE,
           EnumerableRules.ENUMERABLE_VALUES_RULE,
           EnumerableRules.ENUMERABLE_WINDOW_RULE,
-          EnumerableRules.ENUMERABLE_ONE_ROW_RULE,
-          EnumerableRules.ENUMERABLE_EMPTY_RULE,
           SemiJoinRule.INSTANCE,
           TableScanRule.INSTANCE,
           CalcitePrepareImpl.COMMUTE

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 63038fe..0147413 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -35,7 +35,6 @@ import org.apache.calcite.rel.rules.AggregateReduceFunctionsRule;
 import org.apache.calcite.rel.rules.AggregateUnionTransposeRule;
 import org.apache.calcite.rel.rules.CalcMergeRule;
 import org.apache.calcite.rel.rules.CoerceInputsRule;
-import org.apache.calcite.rel.rules.EmptyPruneRules;
 import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
 import org.apache.calcite.rel.rules.FilterJoinRule;
 import org.apache.calcite.rel.rules.FilterProjectTransposeRule;
@@ -52,6 +51,7 @@ import org.apache.calcite.rel.rules.ProjectMergeRule;
 import org.apache.calcite.rel.rules.ProjectRemoveRule;
 import org.apache.calcite.rel.rules.ProjectSetOpTransposeRule;
 import org.apache.calcite.rel.rules.ProjectToCalcRule;
+import org.apache.calcite.rel.rules.PruneEmptyRules;
 import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rel.rules.SemiJoinFilterTransposeRule;
 import org.apache.calcite.rel.rules.SemiJoinJoinTransposeRule;
@@ -520,7 +520,7 @@ public class RelOptRulesTest extends RelOptTestBase {
         .addRuleInstance(ReduceExpressionsRule.CALC_INSTANCE)
 
             // the hard part is done... a few more rule calls to clean up
-        .addRuleInstance(EmptyPruneRules.UNION_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.UNION_INSTANCE)
         .addRuleInstance(ProjectToCalcRule.INSTANCE)
         .addRuleInstance(CalcMergeRule.INSTANCE)
         .addRuleInstance(ReduceExpressionsRule.CALC_INSTANCE)
@@ -745,8 +745,8 @@ public class RelOptRulesTest extends RelOptTestBase {
         .addRuleInstance(FilterProjectTransposeRule.INSTANCE)
         .addRuleInstance(ProjectMergeRule.INSTANCE)
         .addRuleInstance(ValuesReduceRule.PROJECT_FILTER_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.UNION_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.PROJECT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.UNION_INSTANCE)
         .build();
 
     // Plan should be same as for
@@ -763,9 +763,9 @@ public class RelOptRulesTest extends RelOptTestBase {
   @Test public void testEmptyJoin() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.PROJECT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be empty
@@ -778,9 +778,9 @@ public class RelOptRulesTest extends RelOptTestBase {
   @Test public void testEmptyJoinLeft() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.PROJECT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be empty
@@ -793,9 +793,9 @@ public class RelOptRulesTest extends RelOptTestBase {
   @Test public void testEmptyJoinRight() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.PROJECT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_LEFT_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.JOIN_RIGHT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.PROJECT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_LEFT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.JOIN_RIGHT_INSTANCE)
         .build();
 
     // Plan should be equivalent to "select * from emp join dept".
@@ -809,7 +809,7 @@ public class RelOptRulesTest extends RelOptTestBase {
   @Test public void testEmptySort() {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
-        .addRuleInstance(EmptyPruneRules.SORT_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.SORT_INSTANCE)
         .build();
 
     checkPlanning(program,
@@ -818,7 +818,7 @@ public class RelOptRulesTest extends RelOptTestBase {
 
   @Test public void testEmptySortLimitZero() {
     HepProgram program = new HepProgramBuilder()
-        .addRuleInstance(EmptyPruneRules.SORT_FETCH_ZERO_INSTANCE)
+        .addRuleInstance(PruneEmptyRules.SORT_FETCH_ZERO_INSTANCE)
         .build();
 
     checkPlanning(program,

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index d017a36..d4cd7d1 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -476,7 +476,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
-  Empty
+  LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -494,7 +494,7 @@ LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EXPR$0=[$0], EXPR$1=[$1])
-  Empty
+  LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -562,7 +562,7 @@ LogicalProject(X=[+($0, $1)], B=[$1], A=[$0])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-Empty
+LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -718,14 +718,14 @@ LogicalCalc(expr#0=[{inputs}], expr#1=['TABLE'], expr#2=['t'], U=[$t1], S=[$t2])
             <![CDATA[
 LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
   LogicalProject(EXPR$0=[2], EXPR$1=[null])
-    LogicalOneRow
+    LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EXPR$0=[CAST(CASE(IS NULL($1), IS NULL($0), IS NULL($0), IS NULL($1), =($1, $0))):BOOLEAN NOT NULL])
   LogicalProject(EXPR$0=[2], EXPR$1=[null])
-    LogicalOneRow
+    LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
     </TestCase>
@@ -748,7 +748,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  Empty
+  LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -772,7 +772,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
   LogicalJoin(condition=[=($7, $9)], joinType=[right])
-    Empty
+    LogicalValues(tuples=[[]])
     LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
@@ -796,7 +796,7 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
         <Resource name="planAfter">
             <![CDATA[
 LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8], DEPTNO0=[$9], NAME=[$10])
-  Empty
+  LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -833,7 +833,7 @@ Sort(sort0=[$7], dir0=[ASC])
             <![CDATA[
 Sort(sort0=[$7], dir0=[ASC])
   LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$5], COMM=[$6], DEPTNO=[$7], SLACKER=[$8])
-    Empty
+    LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>
@@ -850,7 +850,7 @@ Sort(sort0=[$7], dir0=[ASC], fetch=[0])
         </Resource>
         <Resource name="planAfter">
             <![CDATA[
-Empty
+LogicalValues(tuples=[[]])
 ]]>
         </Resource>
     </TestCase>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index f7036ab..53e7d59 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -20,7 +20,7 @@ limitations under the License.
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[CASE(=('a', 'a'), 1, null)])
-  LogicalOneRow
+  LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -295,9 +295,9 @@ LogicalProject(EXPR$0=['abc'], EXPR$1=[$SLICE($9)])
     Collect(field=[EXPR$0])
       LogicalUnion(all=[true])
         LogicalProject(EXPR$0=[$cor0.DEPTNO])
-          LogicalOneRow
+          LogicalValues(tuples=[[{ 0 }]])
         LogicalProject(EXPR$0=[$cor1.SAL])
-          LogicalOneRow
+          LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -415,7 +415,7 @@ LogicalProject(EXPR$0=[$0])
         Collect(field=[EXPR$0])
           LogicalUnion(all=[true])
             LogicalProject(EXPR$0=[$cor0.DEPTNO])
-              LogicalOneRow
+              LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -564,9 +564,9 @@ LogicalUnion(all=[true])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
   LogicalUnion(all=[true])
     LogicalProject(EXPR$0=[30])
-      LogicalOneRow
+      LogicalValues(tuples=[[{ 0 }]])
     LogicalProject(EXPR$0=[+(45, 10)])
-      LogicalOneRow
+      LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -681,7 +681,7 @@ ProjectRel(EXPR$0=[CASE(IS NULL(1), IS NULL(2), IS NULL(2), IS NULL(1), =(1, 2))
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[NOT(LIKE('a', 'b', 'c'))])
-  LogicalOneRow
+  LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -725,7 +725,7 @@ ProjectRel(EXPR$0=[SUM($5) OVER (PARTITION BY $2 ORDER BY $4 ROWS 2 PRECEDING)],
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EXPR$0=[CHAR_LENGTH('foo')])
-  LogicalOneRow
+  LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
         <Resource name="sql">
@@ -1800,7 +1800,7 @@ LogicalProject(DEPTNO=[$0], NAME=[$1], EXPR$0=[$2])
         Collect(field=[EXPR$0])
           LogicalUnion(all=[true])
             LogicalProject(EXPR$0=[*($cor0.DEPTNO, 2)])
-              LogicalOneRow
+              LogicalValues(tuples=[[{ 0 }]])
 ]]>
         </Resource>
     </TestCase>

http://git-wip-us.apache.org/repos/asf/incubator-calcite/blob/71dd68de/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
----------------------------------------------------------------------
diff --git a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
index 4bd0304..e95dc50 100644
--- a/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
+++ b/spark/src/main/java/org/apache/calcite/adapter/spark/SparkRules.java
@@ -151,7 +151,7 @@ public abstract class SparkRules {
     SparkValues(
         RelOptCluster cluster,
         RelDataType rowType,
-        List<List<RexLiteral>> tuples,
+        ImmutableList<ImmutableList<RexLiteral>> tuples,
         RelTraitSet traitSet) {
       super(cluster, rowType, tuples, traitSet);
     }