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/08/14 22:00:49 UTC

[1/3] git commit: Add class Strong, for detecting null-rejecting predicates.

Repository: incubator-optiq
Updated Branches:
  refs/heads/master 5d209a509 -> 111947cd0


Add class Strong, for detecting null-rejecting predicates.


Project: http://git-wip-us.apache.org/repos/asf/incubator-optiq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-optiq/commit/04e44c2c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-optiq/tree/04e44c2c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-optiq/diff/04e44c2c

Branch: refs/heads/master
Commit: 04e44c2cc31c6348833ddf07545476c3d715a0bb
Parents: 5d209a5
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Aug 12 14:43:04 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 12 14:43:04 2014 -0700

----------------------------------------------------------------------
 .../java/org/eigenbase/relopt/RelOptUtil.java   |  33 +-----
 .../main/java/org/eigenbase/relopt/Strong.java  | 103 +++++++++++++++++++
 .../java/org/eigenbase/test/RexProgramTest.java |  67 ++++++++++++
 3 files changed, 172 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/04e44c2c/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
index a8bfcbe..6bd4571 100644
--- a/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
+++ b/core/src/main/java/org/eigenbase/relopt/RelOptUtil.java
@@ -1957,7 +1957,7 @@ public abstract class RelOptUtil {
         if (smart
             && joinType != null
             && joinType.generatesNullsOnRight()
-            && inputFinder.strongBitSet.intersects(rightBitmap)) {
+            && Strong.is(filter, rightBitmap)) {
           filtersToRemove.add(filter);
           joinType = joinType.cancelNullsOnRight();
           joinTypeHolder.set(joinType);
@@ -1968,7 +1968,7 @@ public abstract class RelOptUtil {
         if (smart
             && joinType != null
             && joinType.generatesNullsOnLeft()
-            && inputFinder.strongBitSet.intersects(leftBitmap)) {
+            && Strong.is(filter, leftBitmap)) {
           filtersToRemove.add(filter);
           joinType = joinType.cancelNullsOnLeft();
           joinTypeHolder.set(joinType);
@@ -2522,7 +2522,6 @@ public abstract class RelOptUtil {
    */
   public static class InputFinder extends RexVisitorImpl<Void> {
     final BitSet inputBitSet;
-    final BitSet strongBitSet = new BitSet();
     private final Set<RelDataTypeField> extraFields;
 
     public InputFinder(BitSet inputBitSet) {
@@ -2539,37 +2538,9 @@ public abstract class RelOptUtil {
     public static InputFinder analyze(RexNode node) {
       final InputFinder inputFinder = new InputFinder(new BitSet());
       node.accept(inputFinder);
-      inputFinder.strong(node);
       return inputFinder;
     }
 
-    private byte strong(RexNode node) {
-      switch (node.getKind()) {
-      case IS_TRUE:
-      case IS_NOT_NULL:
-      case AND:
-      case EQUALS:
-      case NOT_EQUALS:
-      case LESS_THAN:
-      case LESS_THAN_OR_EQUAL:
-      case GREATER_THAN:
-      case GREATER_THAN_OR_EQUAL:
-        return strong(((RexCall) node).getOperands());
-      case INPUT_REF:
-        strongBitSet.set(((RexInputRef) node).getIndex());
-        return 0;
-      default:
-        return 0;
-      }
-    }
-
-    private byte strong(List<RexNode> operands) {
-      for (RexNode operand : operands) {
-        strong(operand);
-      }
-      return 0;
-    }
-
     /**
      * Returns a bit set describing the inputs used by an expression.
      */

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/04e44c2c/core/src/main/java/org/eigenbase/relopt/Strong.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/relopt/Strong.java b/core/src/main/java/org/eigenbase/relopt/Strong.java
new file mode 100644
index 0000000..265090b
--- /dev/null
+++ b/core/src/main/java/org/eigenbase/relopt/Strong.java
@@ -0,0 +1,103 @@
+/*
+// 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.eigenbase.relopt;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.eigenbase.rex.RexCall;
+import org.eigenbase.rex.RexInputRef;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+
+/** Utilities for strong predicates.
+ *
+ * <p>A predicate is strong (or null-rejecting) if it is UNKNOWN if any of its
+ * inputs is UNKNOWN.</p>
+ *
+ * <p>By the way, UNKNOWN is just the boolean form of NULL.</p>
+ *
+ * <p>Examples:</p>
+ * <ul>
+ *   <li>{@code UNKNOWN} is strong
+ *   <li>{@code c = 1} is strong
+ *   <li>{@code c IS NULL} is not strong. (It always returns TRUE or FALSE.)
+ *   <li>{@code p1 AND p2} is strong if p1 or p2 are strong
+ *   <li>{@code p1 OR p2} is strong if p1 and p2 are strong
+ *   <li>{@code c1 = 1 OR c2 IS NULL} is strong on c1 but not c2
+ * </ul>
+ */
+public class Strong {
+  private final BitSet nullColumns;
+
+  private Strong(BitSet nullColumns) {
+    this.nullColumns = nullColumns;
+  }
+
+  public static Strong of(BitSet nullColumns) {
+    return new Strong(nullColumns);
+  }
+
+  /** Returns whether the analyzed expression will return null if a given set
+   * of input columns are null. */
+  public static boolean is(RexNode node, BitSet nullColumns) {
+    return of(nullColumns).strong(node);
+  }
+
+  private boolean strong(RexNode node) {
+    switch (node.getKind()) {
+    case LITERAL:
+      return ((RexLiteral) node).getValue() == null;
+    case IS_TRUE:
+    case IS_NOT_NULL:
+    case AND:
+    case EQUALS:
+    case NOT_EQUALS:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+      return anyStrong(((RexCall) node).getOperands());
+    case OR:
+      return allStrong(((RexCall) node).getOperands());
+    case INPUT_REF:
+      return nullColumns.get(((RexInputRef) node).getIndex());
+    default:
+      return false;
+    }
+  }
+
+  private boolean allStrong(List<RexNode> operands) {
+    for (RexNode operand : operands) {
+      if (!strong(operand)) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+  private boolean anyStrong(List<RexNode> operands) {
+    for (RexNode operand : operands) {
+      if (strong(operand)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}
+
+// End Strong.java

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/04e44c2c/core/src/test/java/org/eigenbase/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/RexProgramTest.java b/core/src/test/java/org/eigenbase/test/RexProgramTest.java
index cd9e6d2..acd17ee 100644
--- a/core/src/test/java/org/eigenbase/test/RexProgramTest.java
+++ b/core/src/test/java/org/eigenbase/test/RexProgramTest.java
@@ -19,6 +19,7 @@ package org.eigenbase.test;
 import java.math.BigDecimal;
 import java.util.*;
 
+import org.eigenbase.relopt.Strong;
 import org.eigenbase.reltype.*;
 import org.eigenbase.rex.*;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
@@ -27,10 +28,14 @@ import org.eigenbase.util.*;
 
 import net.hydromatic.optiq.impl.java.JavaTypeFactory;
 import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
+import net.hydromatic.optiq.util.BitSets;
 
 import org.junit.Before;
 import org.junit.Test;
 
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+
 /**
  * Unit tests for {@link RexProgram} and
  * {@link org.eigenbase.rex.RexProgramBuilder}.
@@ -247,6 +252,68 @@ public class RexProgramTest {
     }
     return builder;
   }
+
+  static boolean strongIf(RexNode e, BitSet b) {
+    return Strong.is(e, b);
+  }
+
+  /** Unit test for {@link org.eigenbase.relopt.Strong}. */
+  @Test public void testStrong() {
+    final RelDataType intType = typeFactory.createSqlType(SqlTypeName.INTEGER);
+    final RelDataType rowType = typeFactory.builder()
+        .add("a", intType)
+        .add("b", intType)
+        .add("c", intType)
+        .add("d", intType)
+        .build();
+
+    final BitSet c = BitSets.of();
+    final BitSet c0 = BitSets.of(0);
+    final BitSet c1 = BitSets.of(1);
+    final BitSet c01 = BitSets.of(0, 1);
+    final BitSet c13 = BitSets.of(1, 3);
+
+    // input ref
+    final RexInputRef aRef = rexBuilder.makeInputRef(intType, 0);
+    final RexInputRef bRef = rexBuilder.makeInputRef(intType, 1);
+    final RexInputRef cRef = rexBuilder.makeInputRef(intType, 2);
+    final RexInputRef dRef = rexBuilder.makeInputRef(intType, 3);
+
+    assertThat(strongIf(aRef, c0), is(true));
+    assertThat(strongIf(aRef, c1), is(false));
+    assertThat(strongIf(aRef, c01), is(true));
+    assertThat(strongIf(aRef, c13), is(false));
+
+    // literals are strong iff they are always null
+    final RexLiteral trueLiteral = rexBuilder.makeLiteral(true);
+    final RexLiteral falseLiteral = rexBuilder.makeLiteral(false);
+    final RexNode nullLiteral = rexBuilder.makeNullLiteral(SqlTypeName.INTEGER);
+    final RexNode unknownLiteral =
+        rexBuilder.makeNullLiteral(SqlTypeName.BOOLEAN);
+
+    assertThat(strongIf(trueLiteral, c), is(false));
+    assertThat(strongIf(trueLiteral, c13), is(false));
+    assertThat(strongIf(falseLiteral, c13), is(false));
+    assertThat(strongIf(nullLiteral, c), is(true));
+    assertThat(strongIf(nullLiteral, c13), is(true));
+    assertThat(strongIf(unknownLiteral, c13), is(true));
+
+    // AND is strong if one of its arguments is strong
+    final RexNode andUnknownTrue =
+        rexBuilder.makeCall(SqlStdOperatorTable.AND,
+            unknownLiteral, trueLiteral);
+    final RexNode andTrueUnknown =
+        rexBuilder.makeCall(SqlStdOperatorTable.AND,
+            trueLiteral, unknownLiteral);
+    final RexNode andFalseTrue =
+        rexBuilder.makeCall(SqlStdOperatorTable.AND,
+            falseLiteral, trueLiteral);
+
+    assertThat(strongIf(andUnknownTrue, c), is(true));
+    assertThat(strongIf(andTrueUnknown, c), is(true));
+    assertThat(strongIf(andFalseTrue, c), is(false));
+  }
+
 }
 
 // End RexProgramTest.java


[3/3] git commit: [OPTIQ-372] Change LoptOptimizeJoinRule & PushFilterPast* rules to use factory

Posted by jh...@apache.org.
[OPTIQ-372] Change LoptOptimizeJoinRule & PushFilterPast* rules to use factory


Project: http://git-wip-us.apache.org/repos/asf/incubator-optiq/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-optiq/commit/111947cd
Tree: http://git-wip-us.apache.org/repos/asf/incubator-optiq/tree/111947cd
Diff: http://git-wip-us.apache.org/repos/asf/incubator-optiq/diff/111947cd

Branch: refs/heads/master
Commit: 111947cd0a7502307507f9e2389e4df13e678844
Parents: cb2ad3a
Author: John Pullokkaran <jp...@hortonworks.com>
Authored: Thu Aug 14 10:52:57 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Thu Aug 14 12:43:37 2014 -0700

----------------------------------------------------------------------
 .../hydromatic/optiq/impl/jdbc/JdbcRules.java   |  6 +--
 .../hydromatic/optiq/rules/java/JavaRules.java  |  6 +--
 .../main/java/org/eigenbase/rel/FilterRel.java  | 18 +++----
 .../java/org/eigenbase/rel/FilterRelBase.java   |  8 +++
 .../rel/rules/LoptOptimizeJoinRule.java         | 32 +++++++-----
 .../rel/rules/PushFilterPastJoinRule.java       | 35 ++++++++-----
 .../rel/rules/PushFilterPastProjectRule.java    | 54 ++++++++++++++------
 .../rel/rules/PushFilterPastSetOpRule.java      | 16 +++---
 .../optiq/impl/mongodb/MongoFilterRel.java      | 10 ++--
 9 files changed, 114 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
index 3fb3175..76c19b1 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
@@ -474,9 +474,9 @@ public class JdbcRules {
       assert getConvention() instanceof JdbcConvention;
     }
 
-    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new JdbcFilterRel(getCluster(), traitSet, sole(inputs),
-          condition);
+    public JdbcFilterRel copy(RelTraitSet traitSet, RelNode input,
+        RexNode condition) {
+      return new JdbcFilterRel(getCluster(), traitSet, input, condition);
     }
 
     public JdbcImplementor.Result implement(JdbcImplementor implementor) {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java b/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
index e6756cb..43d94a8 100644
--- a/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
+++ b/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
@@ -552,9 +552,9 @@ public class JavaRules {
       assert getConvention() instanceof EnumerableConvention;
     }
 
-    @Override public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-      return new EnumerableFilterRel(getCluster(), traitSet, sole(inputs),
-          condition);
+    public EnumerableFilterRel copy(RelTraitSet traitSet, RelNode input,
+        RexNode condition) {
+      return new EnumerableFilterRel(getCluster(), traitSet, input, condition);
     }
 
     public Result implement(EnumerableRelImplementor implementor, Prefer pref) {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/FilterRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/FilterRel.java b/core/src/main/java/org/eigenbase/rel/FilterRel.java
index d652cb5..5acc000 100644
--- a/core/src/main/java/org/eigenbase/rel/FilterRel.java
+++ b/core/src/main/java/org/eigenbase/rel/FilterRel.java
@@ -16,15 +16,16 @@
 */
 package org.eigenbase.rel;
 
-import java.util.List;
-
 import org.eigenbase.relopt.*;
 import org.eigenbase.rex.*;
 
 /**
- * A <code>FilterRel</code> is a relational expression which iterates over its
- * input, and returns elements for which <code>condition</code> evaluates to
+ * Relational expression that iterates over its input
+ * and returns elements for which <code>condition</code> evaluates to
  * <code>true</code>.
+ *
+ * <p>If the condition allows nulls, then a null value is treated the same as
+ * false.</p>
  */
 public final class FilterRel extends FilterRelBase {
   //~ Constructors -----------------------------------------------------------
@@ -58,13 +59,10 @@ public final class FilterRel extends FilterRelBase {
 
   //~ Methods ----------------------------------------------------------------
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  public FilterRel copy(RelTraitSet traitSet, RelNode input,
+      RexNode condition) {
     assert traitSet.containsIfApplicable(Convention.NONE);
-    return new FilterRel(
-        getCluster(),
-        sole(inputs),
-        getCondition());
+    return new FilterRel(getCluster(), input, condition);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/FilterRelBase.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/FilterRelBase.java b/core/src/main/java/org/eigenbase/rel/FilterRelBase.java
index 55776b0..2881621 100644
--- a/core/src/main/java/org/eigenbase/rel/FilterRelBase.java
+++ b/core/src/main/java/org/eigenbase/rel/FilterRelBase.java
@@ -68,6 +68,14 @@ public abstract class FilterRelBase extends SingleRel {
   //~ Methods ----------------------------------------------------------------
 
   @Override
+  public final RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return copy(traitSet, sole(inputs), getCondition());
+  }
+
+  public abstract FilterRelBase copy(RelTraitSet traitSet, RelNode input,
+      RexNode condition);
+
+  @Override
   public List<RexNode> getChildExps() {
     return ImmutableList.of(condition);
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/rules/LoptOptimizeJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/LoptOptimizeJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/LoptOptimizeJoinRule.java
index d9b5fc3..31f104f 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/LoptOptimizeJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/LoptOptimizeJoinRule.java
@@ -41,14 +41,24 @@ import com.google.common.collect.Lists;
  */
 public class LoptOptimizeJoinRule extends RelOptRule {
   public static final LoptOptimizeJoinRule INSTANCE =
-      new LoptOptimizeJoinRule(RelFactories.DEFAULT_JOIN_FACTORY);
+      new LoptOptimizeJoinRule(
+          RelFactories.DEFAULT_JOIN_FACTORY,
+          RelFactories.DEFAULT_PROJECT_FACTORY,
+          RelFactories.DEFAULT_FILTER_FACTORY);
 
   private final RelFactories.JoinFactory joinFactory;
+  private final RelFactories.ProjectFactory projectFactory;
+  private final RelFactories.FilterFactory filterFactory;
 
   /** Creates a LoptOptimizeJoinRule. */
-  public LoptOptimizeJoinRule(RelFactories.JoinFactory joinFactory) {
+  public LoptOptimizeJoinRule(
+      RelFactories.JoinFactory joinFactory,
+      RelFactories.ProjectFactory projectFactory,
+      RelFactories.FilterFactory filterFactory) {
     super(operand(MultiJoinRel.class, any()));
     this.joinFactory = joinFactory;
+    this.projectFactory = projectFactory;
+    this.filterFactory = filterFactory;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -499,9 +509,8 @@ public class LoptOptimizeJoinRule extends RelOptRule {
                 newOffset));
       }
     }
-
-    ProjectRel newProject =
-        (ProjectRel) CalcRel.createProject(
+    ProjectRelBase newProject =
+        (ProjectRelBase) projectFactory.createProject(
             joinTree.getJoinTree(),
             newProjExprs,
             fieldNames);
@@ -511,7 +520,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
     RexNode postJoinFilter =
         multiJoin.getMultiJoinRel().getPostJoinFilter();
     if (postJoinFilter != null) {
-      return CalcRel.createFilter(newProject, postJoinFilter);
+      return filterFactory.createFilter(newProject, postJoinFilter);
     } else {
       return newProject;
     }
@@ -1605,11 +1614,11 @@ public class LoptOptimizeJoinRule extends RelOptRule {
       }
       projects.add(Pair.of(projExpr, newFields.get(i).getName()));
     }
-    ProjectRel projRel =
-        (ProjectRel) CalcRel.createProject(
+    ProjectRelBase projRel =
+        (ProjectRelBase) projectFactory.createProject(
             currJoinRel,
-            projects,
-            false);
+            Pair.left(projects),
+            Pair.right(projects));
 
     // remove the join conditions corresponding to the join we're removing;
     // we don't actually need to use them, but we need to remove them
@@ -1792,8 +1801,7 @@ public class LoptOptimizeJoinRule extends RelOptRule {
                     joinTree.getRowType().getFieldList(),
                     adjustments));
       }
-      joinTree = CalcRel.createFilter(joinTree, filterCond);
-      return joinTree;
+      return filterFactory.createFilter(joinTree, filterCond);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastJoinRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastJoinRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastJoinRule.java
index f97cda6..a742643 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastJoinRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastJoinRule.java
@@ -41,25 +41,29 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
       new PushFilterIntoJoinRule(false);
 
   public static final PushFilterPastJoinRule JOIN =
-      new PushDownJoinConditionRule();
+      new PushDownJoinConditionRule(RelFactories.DEFAULT_FILTER_FACTORY);
 
   /** Whether to try to strengthen join-type. */
   private final boolean smart;
 
+  private final RelFactories.FilterFactory filterFactory;
+
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates a PushFilterPastJoinRule with an explicit root operand.
+   * Creates a PushFilterPastJoinRule with an explicit root operand and
+   * factories.
    */
-  private PushFilterPastJoinRule(RelOptRuleOperand operand, String id,
-      boolean smart) {
+  protected PushFilterPastJoinRule(RelOptRuleOperand operand, String id,
+      boolean smart, RelFactories.FilterFactory filterFactory) {
     super(operand, "PushFilterRule: " + id);
     this.smart = smart;
+    this.filterFactory = filterFactory;
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  protected void perform(RelOptRuleCall call, FilterRel filter,
+  protected void perform(RelOptRuleCall call, FilterRelBase filter,
       JoinRelBase join) {
     final List<RexNode> joinFilters =
         RelOptUtil.conjunctions(join.getCondition());
@@ -217,16 +221,16 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
     if (andFilters.isAlwaysTrue()) {
       return rel;
     }
-    return CalcRel.createFilter(rel, andFilters);
+    return filterFactory.createFilter(rel, andFilters);
   }
 
   /** Rule that pushes parts of the join condition to its inputs. */
-  private static class PushDownJoinConditionRule
+  public static class PushDownJoinConditionRule
       extends PushFilterPastJoinRule {
-    public PushDownJoinConditionRule() {
+    public PushDownJoinConditionRule(RelFactories.FilterFactory filterFactory) {
       super(RelOptRule.operand(JoinRelBase.class, RelOptRule.any()),
           "PushFilterPastJoinRule:no-filter",
-          true);
+          true, filterFactory);
     }
 
     @Override
@@ -238,18 +242,23 @@ public abstract class PushFilterPastJoinRule extends RelOptRule {
 
   /** Rule that tries to push filter expressions into a join
    * condition and into the inputs of the join. */
-  private static class PushFilterIntoJoinRule extends PushFilterPastJoinRule {
+  public static class PushFilterIntoJoinRule extends PushFilterPastJoinRule {
     public PushFilterIntoJoinRule(boolean smart) {
+      this(smart, RelFactories.DEFAULT_FILTER_FACTORY);
+    }
+
+    public PushFilterIntoJoinRule(boolean smart,
+        RelFactories.FilterFactory filterFactory) {
       super(
-          RelOptRule.operand(FilterRel.class,
+          RelOptRule.operand(FilterRelBase.class,
               RelOptRule.operand(JoinRelBase.class, RelOptRule.any())),
           "PushFilterPastJoinRule:filter",
-          smart);
+          smart, filterFactory);
     }
 
     @Override
     public void onMatch(RelOptRuleCall call) {
-      FilterRel filter = call.rel(0);
+      FilterRelBase filter = call.rel(0);
       JoinRelBase join = call.rel(1);
       perform(call, filter, join);
     }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastProjectRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastProjectRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastProjectRule.java
index 9f37442..53efdc8 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastProjectRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastProjectRule.java
@@ -25,43 +25,63 @@ import org.eigenbase.rex.*;
  * past a {@link ProjectRel}.
  */
 public class PushFilterPastProjectRule extends RelOptRule {
+  /** The default instance of
+   * {@link org.eigenbase.rel.rules.PushFilterPastJoinRule}.
+   *
+   * <p>It matches any kind of join or filter, and generates the same kind of
+   * join and filter. It uses null values for {@code filterFactory} and
+   * {@code projectFactory} to achieve this. */
   public static final PushFilterPastProjectRule INSTANCE =
-      new PushFilterPastProjectRule();
+      new PushFilterPastProjectRule(
+          FilterRelBase.class, null,
+          ProjectRelBase.class, null);
+
+  private final RelFactories.FilterFactory filterFactory;
+  private final RelFactories.ProjectFactory projectFactory;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a PushFilterPastProjectRule.
+   *
+   * <p>If {@code filterFactory} is null, creates the same kind of filter as
+   * matched in the rule. Similarly {@code projectFactory}.</p>
    */
-  private PushFilterPastProjectRule() {
+  public PushFilterPastProjectRule(
+      Class<? extends FilterRelBase> filterClass,
+      RelFactories.FilterFactory filterFactory,
+      Class<? extends ProjectRelBase> projectRelBaseClass,
+      RelFactories.ProjectFactory projectFactory) {
     super(
-        operand(
-            FilterRel.class,
-            operand(ProjectRel.class, any())));
+        operand(filterClass,
+            operand(projectRelBaseClass, any())));
+    this.filterFactory = filterFactory;
+    this.projectFactory = projectFactory;
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    FilterRel filterRel = call.rel(0);
-    ProjectRel projRel = call.rel(1);
+    final FilterRelBase filterRel = call.rel(0);
+    final ProjectRelBase projRel = call.rel(1);
 
     // convert the filter to one that references the child of the project
     RexNode newCondition =
         RelOptUtil.pushFilterPastProject(filterRel.getCondition(), projRel);
 
-    FilterRel newFilterRel =
-        new FilterRel(
-            filterRel.getCluster(),
-            projRel.getChild(),
-            newCondition);
+    RelNode newFilterRel =
+        filterFactory == null
+            ? filterRel.copy(filterRel.getTraitSet(), projRel.getChild(),
+                newCondition)
+            : filterFactory.createFilter(projRel.getChild(), newCondition);
 
-    ProjectRel newProjRel =
-        (ProjectRel) CalcRel.createProject(
-            newFilterRel,
-            projRel.getNamedProjects(),
-            false);
+    RelNode newProjRel =
+        projectFactory == null
+            ? projRel.copy(projRel.getTraitSet(), newFilterRel,
+                projRel.getProjects(), projRel.getRowType())
+            : projectFactory.createProject(newFilterRel, projRel.getProjects(),
+                projRel.getRowType().getFieldNames());
 
     call.transformTo(newProjRel);
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastSetOpRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastSetOpRule.java b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastSetOpRule.java
index f3165db..6743a3c 100644
--- a/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastSetOpRule.java
+++ b/core/src/main/java/org/eigenbase/rel/rules/PushFilterPastSetOpRule.java
@@ -30,28 +30,29 @@ import org.eigenbase.rex.*;
  */
 public class PushFilterPastSetOpRule extends RelOptRule {
   public static final PushFilterPastSetOpRule INSTANCE =
-      new PushFilterPastSetOpRule();
+      new PushFilterPastSetOpRule(RelFactories.DEFAULT_FILTER_FACTORY);
+
+  private final RelFactories.FilterFactory filterFactory;
 
   //~ Constructors -----------------------------------------------------------
 
   /**
    * Creates a PushFilterPastSetOpRule.
    */
-  private PushFilterPastSetOpRule() {
+  private PushFilterPastSetOpRule(RelFactories.FilterFactory filterFactory) {
     super(
-        operand(
-            FilterRel.class,
+        operand(FilterRelBase.class,
             operand(SetOpRel.class, any())));
+    this.filterFactory = filterFactory;
   }
 
   //~ Methods ----------------------------------------------------------------
 
   // implement RelOptRule
   public void onMatch(RelOptRuleCall call) {
-    FilterRel filterRel = call.rel(0);
+    FilterRelBase filterRel = call.rel(0);
     SetOpRel setOpRel = call.rel(1);
 
-    RelOptCluster cluster = setOpRel.getCluster();
     RexNode condition = filterRel.getCondition();
 
     // create filters on top of each setop child, modifying the filter
@@ -69,8 +70,7 @@ public class PushFilterPastSetOpRule extends RelOptRule {
                   origFields,
                   input.getRowType().getFieldList(),
                   adjustments));
-      newSetOpInputs.add(
-          new FilterRel(cluster, input, newCondition));
+      newSetOpInputs.add(filterFactory.createFilter(input, newCondition));
     }
 
     // create a new setop whose children are the filters created above

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/111947cd/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoFilterRel.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoFilterRel.java b/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoFilterRel.java
index ad20562..c36041a 100644
--- a/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoFilterRel.java
+++ b/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoFilterRel.java
@@ -28,8 +28,8 @@ import com.google.common.collect.Multimap;
 import java.util.*;
 
 /**
- * Implementation of {@link org.eigenbase.rel.FilterRel} relational expression in
- * MongoDB.
+ * Implementation of a {@link org.eigenbase.rel.FilterRel} relational expression
+ * in MongoDB.
  */
 public class MongoFilterRel
     extends FilterRelBase
@@ -49,9 +49,9 @@ public class MongoFilterRel
     return super.computeSelfCost(planner).multiplyBy(0.1);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new MongoFilterRel(getCluster(), traitSet, sole(inputs), condition);
+  public MongoFilterRel copy(RelTraitSet traitSet, RelNode input,
+      RexNode condition) {
+    return new MongoFilterRel(getCluster(), traitSet, input, condition);
   }
 
   public void implement(Implementor implementor) {


[2/3] git commit: Upgrade maven-checkstyle-plugin.

Posted by jh...@apache.org.
Upgrade maven-checkstyle-plugin.


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

Branch: refs/heads/master
Commit: cb2ad3acec33c4e17c54b9acbae54fbfe500a84d
Parents: 04e44c2
Author: Julian Hyde <jh...@apache.org>
Authored: Tue Aug 12 15:03:02 2014 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Aug 12 15:03:02 2014 -0700

----------------------------------------------------------------------
 .../java/net/hydromatic/avatica/ArrayImpl.java  |  4 +--
 .../hydromatic/avatica/AvaticaConnection.java   |  2 +-
 .../avatica/AvaticaDatabaseMetaData.java        | 10 +++----
 .../hydromatic/avatica/ConnectStringParser.java | 12 ++++-----
 .../net/hydromatic/avatica/HandlerImpl.java     |  2 +-
 .../hydromatic/optiq/impl/jdbc/JdbcRules.java   |  4 +--
 .../hydromatic/optiq/impl/jdbc/JdbcUtils.java   |  2 +-
 .../java/net/hydromatic/optiq/jdbc/Driver.java  |  2 +-
 .../optiq/jdbc/OptiqPreparedStatement.java      |  2 +-
 .../hydromatic/optiq/model/ModelHandler.java    |  2 +-
 .../hydromatic/optiq/rules/java/JavaRules.java  |  4 +--
 .../net/hydromatic/optiq/tools/Planner.java     |  2 +-
 .../java/net/hydromatic/optiq/tools/SqlRun.java |  8 +++---
 .../net/hydromatic/optiq/util/Compatible.java   |  2 +-
 .../org/eigenbase/javac/JaninoCompiler.java     |  2 +-
 .../metadata/CachingRelMetadataProvider.java    |  2 +-
 .../metadata/ChainedRelMetadataProvider.java    |  2 +-
 .../org/eigenbase/util/SerializableCharset.java |  2 +-
 .../net/hydromatic/optiq/test/JdbcTest.java     | 28 ++++++++++----------
 .../net/hydromatic/optiq/test/ModelTest.java    |  2 +-
 .../optiq/test/MultiJdbcSchemaJoinTest.java     |  4 +--
 .../net/hydromatic/optiq/test/OptiqAssert.java  | 16 +++++------
 .../net/hydromatic/optiq/tools/PlannerTest.java |  4 +--
 .../java/org/eigenbase/test/DiffTestCase.java   |  2 +-
 .../org/eigenbase/test/RelOptRulesTest.java     | 12 ++++-----
 .../ConcurrentTestCommandGenerator.java         | 22 +++++++--------
 .../concurrent/ConcurrentTestCommandScript.java | 24 ++++++++---------
 .../optiq/impl/mongodb/MongoAggregateRel.java   |  2 +-
 pom.xml                                         |  2 +-
 .../splunk/search/SplunkConnectionImpl.java     |  4 +--
 .../optiq/impl/splunk/util/HttpUtils.java       |  2 +-
 .../optiq/test/SplunkAdapterTest.java           |  2 +-
 src/main/config/checkstyle/suppressions.xml     |  3 +++
 .../net/hydromatic/optiq/StatementTest.java     |  2 +-
 34 files changed, 100 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/avatica/src/main/java/net/hydromatic/avatica/ArrayImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/net/hydromatic/avatica/ArrayImpl.java b/avatica/src/main/java/net/hydromatic/avatica/ArrayImpl.java
index 8de6368..55314ee 100644
--- a/avatica/src/main/java/net/hydromatic/avatica/ArrayImpl.java
+++ b/avatica/src/main/java/net/hydromatic/avatica/ArrayImpl.java
@@ -133,7 +133,7 @@ public class ArrayImpl implements Array {
   }
 
   public Object getArray(long index, int count, Map<String, Class<?>> map)
-    throws SQLException {
+      throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 
@@ -142,7 +142,7 @@ public class ArrayImpl implements Array {
   }
 
   public ResultSet getResultSet(Map<String, Class<?>> map)
-    throws SQLException {
+      throws SQLException {
     throw new UnsupportedOperationException(); // TODO
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/avatica/src/main/java/net/hydromatic/avatica/AvaticaConnection.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/net/hydromatic/avatica/AvaticaConnection.java b/avatica/src/main/java/net/hydromatic/avatica/AvaticaConnection.java
index 8403f7b..9b9994d 100644
--- a/avatica/src/main/java/net/hydromatic/avatica/AvaticaConnection.java
+++ b/avatica/src/main/java/net/hydromatic/avatica/AvaticaConnection.java
@@ -303,7 +303,7 @@ public abstract class AvaticaConnection implements Connection {
   }
 
   public void setClientInfo(Properties properties)
-    throws SQLClientInfoException {
+      throws SQLClientInfoException {
     throw new UnsupportedOperationException();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/avatica/src/main/java/net/hydromatic/avatica/AvaticaDatabaseMetaData.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/net/hydromatic/avatica/AvaticaDatabaseMetaData.java b/avatica/src/main/java/net/hydromatic/avatica/AvaticaDatabaseMetaData.java
index 0e995a0..b973f14 100644
--- a/avatica/src/main/java/net/hydromatic/avatica/AvaticaDatabaseMetaData.java
+++ b/avatica/src/main/java/net/hydromatic/avatica/AvaticaDatabaseMetaData.java
@@ -517,17 +517,17 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
   }
 
   public boolean supportsTransactionIsolationLevel(int level)
-    throws SQLException {
+      throws SQLException {
     return level == Connection.TRANSACTION_NONE;
   }
 
   public boolean supportsDataDefinitionAndDataManipulationTransactions()
-    throws SQLException {
+      throws SQLException {
     return false;
   }
 
   public boolean supportsDataManipulationTransactionsOnly()
-    throws SQLException {
+      throws SQLException {
     return true;
   }
 
@@ -776,7 +776,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
   }
 
   public boolean supportsResultSetHoldability(int holdability)
-    throws SQLException {
+      throws SQLException {
     throw connection.helper.todo();
   }
 
@@ -817,7 +817,7 @@ public class AvaticaDatabaseMetaData implements DatabaseMetaData {
   }
 
   public boolean supportsStoredFunctionsUsingCallSyntax()
-    throws SQLException {
+      throws SQLException {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/avatica/src/main/java/net/hydromatic/avatica/ConnectStringParser.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/net/hydromatic/avatica/ConnectStringParser.java b/avatica/src/main/java/net/hydromatic/avatica/ConnectStringParser.java
index e522c0f..00a3d35 100644
--- a/avatica/src/main/java/net/hydromatic/avatica/ConnectStringParser.java
+++ b/avatica/src/main/java/net/hydromatic/avatica/ConnectStringParser.java
@@ -92,7 +92,7 @@ public class ConnectStringParser {
    * @throws SQLException error parsing name-value pairs
    */
   public static Properties parse(String s)
-    throws SQLException {
+      throws SQLException {
     return new ConnectStringParser(s).parseInternal(null);
   }
 
@@ -109,7 +109,7 @@ public class ConnectStringParser {
    * @throws SQLException error parsing name-value pairs
    */
   public static Properties parse(String s, Properties props)
-    throws SQLException {
+      throws SQLException {
     return new ConnectStringParser(s).parseInternal(props);
   }
 
@@ -127,7 +127,7 @@ public class ConnectStringParser {
    * @throws SQLException error parsing name-value pairs
    */
   Properties parseInternal(Properties props)
-    throws SQLException {
+      throws SQLException {
     if (props == null) {
       props = new Properties();
     }
@@ -143,7 +143,7 @@ public class ConnectStringParser {
    * @throws SQLException error parsing value
    */
   void parsePair(Properties props)
-    throws SQLException {
+      throws SQLException {
     String name = parseName();
     String value;
     if (i >= n) {
@@ -199,7 +199,7 @@ public class ConnectStringParser {
    * @throws SQLException if find an unterminated quoted value
    */
   String parseValue()
-    throws SQLException {
+      throws SQLException {
     char c;
 
     // skip over leading white space
@@ -249,7 +249,7 @@ public class ConnectStringParser {
    * @throws SQLException if find an unterminated quoted value
    */
   String parseQuoted(char q)
-    throws SQLException {
+      throws SQLException {
     char c = s.charAt(i++);
     if (c != q) {
       throw new AssertionError("c != q: c=" + c + " q=" + q);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/avatica/src/main/java/net/hydromatic/avatica/HandlerImpl.java
----------------------------------------------------------------------
diff --git a/avatica/src/main/java/net/hydromatic/avatica/HandlerImpl.java b/avatica/src/main/java/net/hydromatic/avatica/HandlerImpl.java
index 5648465..b70aaf5 100644
--- a/avatica/src/main/java/net/hydromatic/avatica/HandlerImpl.java
+++ b/avatica/src/main/java/net/hydromatic/avatica/HandlerImpl.java
@@ -25,7 +25,7 @@ import java.sql.SQLException;
  */
 public class HandlerImpl implements Handler {
   public void onConnectionInit(AvaticaConnection connection)
-    throws SQLException {
+      throws SQLException {
     // nothing
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
index b52b804..3fb3175 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcRules.java
@@ -188,7 +188,7 @@ public class JdbcRules {
         ImmutableIntList rightKeys,
         JoinRelType joinType,
         Set<String> variablesStopped)
-      throws InvalidRelException {
+        throws InvalidRelException {
       super(cluster, traits, left, right, condition, leftKeys, rightKeys,
           joinType, variablesStopped);
     }
@@ -521,7 +521,7 @@ public class JdbcRules {
         RelNode child,
         BitSet groupSet,
         List<AggregateCall> aggCalls)
-      throws InvalidRelException {
+        throws InvalidRelException {
       super(cluster, traitSet, child, groupSet, aggCalls);
       assert getConvention() instanceof JdbcConvention;
     }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcUtils.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcUtils.java b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcUtils.java
index 27c30e1..a329a60 100644
--- a/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcUtils.java
+++ b/core/src/main/java/net/hydromatic/optiq/impl/jdbc/JdbcUtils.java
@@ -92,7 +92,7 @@ final class JdbcUtils {
 
     public ObjectArrayRowBuilder(
         ResultSet resultSet, Primitive[] primitives, int[] types)
-      throws SQLException {
+        throws SQLException {
       this.resultSet = resultSet;
       this.primitives = primitives;
       this.types = types;

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/jdbc/Driver.java b/core/src/main/java/net/hydromatic/optiq/jdbc/Driver.java
index 22e921f..63b2db0 100644
--- a/core/src/main/java/net/hydromatic/optiq/jdbc/Driver.java
+++ b/core/src/main/java/net/hydromatic/optiq/jdbc/Driver.java
@@ -82,7 +82,7 @@ public class Driver extends UnregisteredDriver {
     return new HandlerImpl() {
       @Override
       public void onConnectionInit(AvaticaConnection connection_)
-        throws SQLException {
+          throws SQLException {
         final OptiqConnectionImpl connection =
             (OptiqConnectionImpl) connection_;
         super.onConnectionInit(connection);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/jdbc/OptiqPreparedStatement.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/jdbc/OptiqPreparedStatement.java b/core/src/main/java/net/hydromatic/optiq/jdbc/OptiqPreparedStatement.java
index 05f362d..8d9906e 100644
--- a/core/src/main/java/net/hydromatic/optiq/jdbc/OptiqPreparedStatement.java
+++ b/core/src/main/java/net/hydromatic/optiq/jdbc/OptiqPreparedStatement.java
@@ -47,7 +47,7 @@ abstract class OptiqPreparedStatement
       int resultSetType,
       int resultSetConcurrency,
       int resultSetHoldability)
-    throws SQLException {
+      throws SQLException {
     super(
         connection, prepareResult, resultSetType, resultSetConcurrency,
         resultSetHoldability);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/model/ModelHandler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/model/ModelHandler.java b/core/src/main/java/net/hydromatic/optiq/model/ModelHandler.java
index efef49d..be4719a 100644
--- a/core/src/main/java/net/hydromatic/optiq/model/ModelHandler.java
+++ b/core/src/main/java/net/hydromatic/optiq/model/ModelHandler.java
@@ -47,7 +47,7 @@ public class ModelHandler {
       new ArrayList<Pair<String, SchemaPlus>>();
 
   public ModelHandler(OptiqConnection connection, String uri)
-    throws IOException {
+      throws IOException {
     super();
     this.connection = connection;
     final ObjectMapper mapper = new ObjectMapper();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java b/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
index b5027d2..e6756cb 100644
--- a/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
+++ b/core/src/main/java/net/hydromatic/optiq/rules/java/JavaRules.java
@@ -136,7 +136,7 @@ public class JavaRules {
         ImmutableIntList rightKeys,
         JoinRelType joinType,
         Set<String> variablesStopped)
-      throws InvalidRelException {
+        throws InvalidRelException {
       super(
           cluster,
           traits,
@@ -834,7 +834,7 @@ public class JavaRules {
         RelNode child,
         BitSet groupSet,
         List<AggregateCall> aggCalls)
-      throws InvalidRelException {
+        throws InvalidRelException {
       super(cluster, traitSet, child, groupSet, aggCalls);
       assert getConvention() instanceof EnumerableConvention;
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/tools/Planner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/Planner.java b/core/src/main/java/net/hydromatic/optiq/tools/Planner.java
index f572686..a082263 100644
--- a/core/src/main/java/net/hydromatic/optiq/tools/Planner.java
+++ b/core/src/main/java/net/hydromatic/optiq/tools/Planner.java
@@ -83,7 +83,7 @@ public interface Planner {
    */
   RelNode transform(int ruleSetIndex,
       RelTraitSet requiredOutputTraits, RelNode rel)
-    throws RelConversionException;
+      throws RelConversionException;
 
   /**
    * Resets this {@code Planner} to be used with a new query. This

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java b/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
index 09b9c5a..11a4ff4 100644
--- a/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
+++ b/core/src/main/java/net/hydromatic/optiq/tools/SqlRun.java
@@ -297,7 +297,7 @@ public class SqlRun {
       @Override
       public void format(ResultSet resultSet, List<String> headerLines,
           List<String> bodyLines, List<String> footerLines, SqlRun run)
-        throws Exception {
+          throws Exception {
         final ResultSetMetaData metaData = resultSet.getMetaData();
         final PrintWriter pw = run.printWriter;
         final int n = metaData.getColumnCount();
@@ -331,7 +331,7 @@ public class SqlRun {
       @Override
       public void format(ResultSet resultSet, List<String> headerLines,
           List<String> bodyLines, List<String> footerLines, SqlRun run)
-        throws Exception {
+          throws Exception {
         SqlRun.format(resultSet, headerLines, bodyLines, footerLines, run.sort,
             false);
       }
@@ -350,7 +350,7 @@ public class SqlRun {
       @Override
       public void format(ResultSet resultSet, List<String> headerLines,
           List<String> bodyLines, List<String> footerLines, SqlRun run)
-        throws Exception {
+          throws Exception {
         SqlRun.format(resultSet, headerLines, bodyLines, footerLines, run.sort,
             true);
       }
@@ -358,7 +358,7 @@ public class SqlRun {
 
     public abstract void format(ResultSet resultSet, List<String> headerLines,
         List<String> bodyLines, List<String> footerLines, SqlRun run)
-      throws Exception;
+        throws Exception;
   }
 
   private static void format(ResultSet resultSet, List<String> headerLines,

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/net/hydromatic/optiq/util/Compatible.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/net/hydromatic/optiq/util/Compatible.java b/core/src/main/java/net/hydromatic/optiq/util/Compatible.java
index 02adcc5..3317dba 100644
--- a/core/src/main/java/net/hydromatic/optiq/util/Compatible.java
+++ b/core/src/main/java/net/hydromatic/optiq/util/Compatible.java
@@ -42,7 +42,7 @@ public interface Compatible {
           new Class<?>[] {Compatible.class},
           new InvocationHandler() {
             public Object invoke(Object proxy, Method method, Object[] args)
-              throws Throwable {
+                throws Throwable {
               if (method.getName().equals("asMap")) {
                 // Use the Guava implementation Maps.asMap if it is available
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/org/eigenbase/javac/JaninoCompiler.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/javac/JaninoCompiler.java b/core/src/main/java/org/eigenbase/javac/JaninoCompiler.java
index 527e8fa..27cfc05 100644
--- a/core/src/main/java/org/eigenbase/javac/JaninoCompiler.java
+++ b/core/src/main/java/org/eigenbase/javac/JaninoCompiler.java
@@ -156,7 +156,7 @@ public class JaninoCompiler implements JavaCompiler {
 
     // override JavaSourceClassLoader
     public Map generateBytecodes(String name)
-      throws ClassNotFoundException {
+        throws ClassNotFoundException {
       Map<String, byte[]> map = super.generateBytecodes(name);
       if (map == null) {
         return map;

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/org/eigenbase/rel/metadata/CachingRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/metadata/CachingRelMetadataProvider.java b/core/src/main/java/org/eigenbase/rel/metadata/CachingRelMetadataProvider.java
index fe16222..cba69e6 100644
--- a/core/src/main/java/org/eigenbase/rel/metadata/CachingRelMetadataProvider.java
+++ b/core/src/main/java/org/eigenbase/rel/metadata/CachingRelMetadataProvider.java
@@ -101,7 +101,7 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
     }
 
     public Object invoke(Object proxy, Method method, Object[] args)
-      throws Throwable {
+        throws Throwable {
       // Compute hash key.
       final ImmutableList.Builder<Object> builder = ImmutableList.builder();
       builder.add(method);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/org/eigenbase/rel/metadata/ChainedRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/rel/metadata/ChainedRelMetadataProvider.java b/core/src/main/java/org/eigenbase/rel/metadata/ChainedRelMetadataProvider.java
index b420ecb..8e3b49e 100644
--- a/core/src/main/java/org/eigenbase/rel/metadata/ChainedRelMetadataProvider.java
+++ b/core/src/main/java/org/eigenbase/rel/metadata/ChainedRelMetadataProvider.java
@@ -102,7 +102,7 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
     }
 
     public Object invoke(Object proxy, Method method, Object[] args)
-      throws Throwable {
+        throws Throwable {
       for (Metadata metadata : metadataList) {
         final Object o = method.invoke(metadata, args);
         if (o != null) {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/main/java/org/eigenbase/util/SerializableCharset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/eigenbase/util/SerializableCharset.java b/core/src/main/java/org/eigenbase/util/SerializableCharset.java
index e98696d..6037340 100644
--- a/core/src/main/java/org/eigenbase/util/SerializableCharset.java
+++ b/core/src/main/java/org/eigenbase/util/SerializableCharset.java
@@ -62,7 +62,7 @@ public class SerializableCharset implements Serializable {
    * Per {@link Serializable}.
    */
   private void readObject(ObjectInputStream in)
-    throws IOException, ClassNotFoundException {
+      throws IOException, ClassNotFoundException {
     charsetName = (String) in.readObject();
     charset = Charset.availableCharsets().get(this.charsetName);
   }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
index cd3f7a1..c6c98ee 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/JdbcTest.java
@@ -165,7 +165,7 @@ public class JdbcTest {
    * Tests a table function with literal arguments.
    */
   @Test public void testTableFunction()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:optiq:");
@@ -189,7 +189,7 @@ public class JdbcTest {
    * actual call arguments.
    */
   @Test public void testTableFunctionDynamicStructure()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Connection connection = getConnectionWithMultiplyFunction();
     final PreparedStatement ps = connection.prepareStatement(
         "select *\n"
@@ -209,7 +209,7 @@ public class JdbcTest {
    */
   @Ignore("SQLException does not include message from nested exception")
   @Test public void testTableFunctionNonNullableMustBeLiterals()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Connection connection = getConnectionWithMultiplyFunction();
     try {
       final PreparedStatement ps = connection.prepareStatement(
@@ -231,7 +231,7 @@ public class JdbcTest {
   }
 
   private Connection getConnectionWithMultiplyFunction()
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:optiq:");
@@ -251,7 +251,7 @@ public class JdbcTest {
   @Ignore("CannotPlanException: Node [rel#18:Subset#4.ENUMERABLE.[]] "
           + "could not be implemented")
   @Test public void testTableFunctionCursorInputs()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:optiq:");
@@ -288,7 +288,7 @@ public class JdbcTest {
   @Ignore("CannotPlanException: Node [rel#24:Subset#6.ENUMERABLE.[]] "
           + "could not be implemented")
   @Test public void testTableFunctionCursorsInputs()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         getConnectionWithMultiplyFunction();
@@ -329,7 +329,7 @@ public class JdbcTest {
    * Tests {@link org.eigenbase.sql.advise.SqlAdvisorGetHintsFunction}.
    */
   @Test public void testSqlAdvisorGetHintsFunction()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     String res = adviseSql("select e.e^ from \"emps\" e");
     assertThat(res,
         equalTo(
@@ -340,7 +340,7 @@ public class JdbcTest {
    * Tests {@link org.eigenbase.sql.advise.SqlAdvisorGetHintsFunction}.
    */
   @Test public void testSqlAdvisorSchemaNames()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     String res = adviseSql("select empid from \"emps\" e, ^");
     assertThat(res,
         equalTo(
@@ -390,7 +390,7 @@ public class JdbcTest {
    * {@link net.hydromatic.optiq.TranslatableTable}.
    */
   @Test public void testTableMacro()
-    throws SQLException, ClassNotFoundException {
+      throws SQLException, ClassNotFoundException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection =
         DriverManager.getConnection("jdbc:optiq:");
@@ -794,7 +794,7 @@ public class JdbcTest {
 
   /** Tests driver's implementation of {@link DatabaseMetaData#getColumns}. */
   @Test public void testMetaDataColumns()
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     Connection connection = OptiqAssert.getConnection("hr", "foodmart");
     DatabaseMetaData metaData = connection.getMetaData();
     ResultSet resultSet = metaData.getColumns(null, null, null, null);
@@ -814,7 +814,7 @@ public class JdbcTest {
   /** Tests driver's implementation of {@link DatabaseMetaData#getPrimaryKeys}.
    * It is empty but it should still have column definitions. */
   @Test public void testMetaDataPrimaryKeys()
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     Connection connection = OptiqAssert.getConnection("hr", "foodmart");
     DatabaseMetaData metaData = connection.getMetaData();
     ResultSet resultSet = metaData.getPrimaryKeys(null, null, null);
@@ -860,7 +860,7 @@ public class JdbcTest {
 
   /** Tests driver's implementation of {@link DatabaseMetaData#getColumns}. */
   @Test public void testResultSetMetaData()
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     Connection connection = OptiqAssert.getConnection("hr", "foodmart");
     Statement statement = connection.createStatement();
     ResultSet resultSet =
@@ -955,7 +955,7 @@ public class JdbcTest {
   }
 
   @Test public void testCloneSchema()
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     final OptiqConnection connection = OptiqAssert.getConnection(false);
     final SchemaPlus rootSchema = connection.getRootSchema();
     final SchemaPlus foodmart = rootSchema.getSubSchema("foodmart");
@@ -4553,7 +4553,7 @@ public class JdbcTest {
 
   /** Tests that an immutable schema in a model cannot contain a view. */
   @Test public void testModelImmutableSchemaCannotContainView()
-    throws Exception {
+      throws Exception {
     final OptiqAssert.AssertThat that =
         OptiqAssert.that().withModel(
             "{\n"

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java b/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
index f1e4098..a7f45bd 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/ModelTest.java
@@ -142,7 +142,7 @@ public class ModelTest {
   /** Tests that an immutable schema in a model cannot contain a
    * materialization. */
   @Test public void testModelImmutableSchemaCannotContainMaterialization()
-    throws Exception {
+      throws Exception {
     final OptiqAssert.AssertThat that =
         OptiqAssert.that().withModel(
             "{\n"

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/net/hydromatic/optiq/test/MultiJdbcSchemaJoinTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/MultiJdbcSchemaJoinTest.java b/core/src/test/java/net/hydromatic/optiq/test/MultiJdbcSchemaJoinTest.java
index 138cd84..670d3b5 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/MultiJdbcSchemaJoinTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/MultiJdbcSchemaJoinTest.java
@@ -126,7 +126,7 @@ public class MultiJdbcSchemaJoinTest {
   }
 
   @Test public void testEnumerableWithJdbcJoinWithWhereClause()
-    throws SQLException {
+      throws SQLException {
     // Same query as above but with a where condition added:
     //  * the good: this query does not give a CannotPlanException
     //  * the bad: the result is wrong: there is only one emp called Bill.
@@ -142,7 +142,7 @@ public class MultiJdbcSchemaJoinTest {
   }
 
   private Set<Integer> runQuery(Connection optiqConn, String query)
-    throws SQLException {
+      throws SQLException {
     // Print out the plan
     Statement stmt = optiqConn.createStatement();
     try {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java b/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
index aa44dfb..b236144 100644
--- a/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
+++ b/core/src/test/java/net/hydromatic/optiq/test/OptiqAssert.java
@@ -131,7 +131,7 @@ public class OptiqAssert {
 
         @Override
         public <T> AssertThat doWithConnection(Function1<OptiqConnection, T> fn)
-          throws Exception {
+            throws Exception {
           return this;
         }
 
@@ -333,7 +333,7 @@ public class OptiqAssert {
   }
 
   private static String typeString(ResultSetMetaData metaData)
-    throws SQLException {
+      throws SQLException {
     final List<String> list = new ArrayList<String>();
     for (int i = 0; i < metaData.getColumnCount(); i++) {
       list.add(
@@ -481,7 +481,7 @@ public class OptiqAssert {
   }
 
   static ImmutableMultiset<String> toSet(ResultSet resultSet)
-    throws SQLException {
+      throws SQLException {
     return ImmutableMultiset.copyOf(
         toStringList(resultSet, new ArrayList<String>()));
   }
@@ -489,7 +489,7 @@ public class OptiqAssert {
   /** Calls a non-static method via reflection. Useful for testing methods that
    * don't exist in certain versions of the JDK. */
   static Object call(Object o, String methodName, Object... args)
-    throws NoSuchMethodException, InvocationTargetException,
+      throws NoSuchMethodException, InvocationTargetException,
       IllegalAccessException {
     return method(o, methodName, args).invoke(o, args);
   }
@@ -595,7 +595,7 @@ public class OptiqAssert {
   }
 
   static OptiqConnection getConnection(String... schema)
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     final List<String> schemaList = Arrays.asList(schema);
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     String suffix = schemaList.contains("spark") ? "spark=true" : "";
@@ -635,7 +635,7 @@ public class OptiqAssert {
    * @throws java.sql.SQLException
    */
   static OptiqConnection getConnection(boolean withClone)
-    throws ClassNotFoundException, SQLException {
+      throws ClassNotFoundException, SQLException {
     Class.forName("net.hydromatic.optiq.jdbc.Driver");
     Connection connection = DriverManager.getConnection("jdbc:optiq:");
     OptiqConnection optiqConnection =
@@ -786,7 +786,7 @@ public class OptiqAssert {
 
     /** Creates a {@link OptiqConnection} and executes a callback. */
     public <T> AssertThat doWithConnection(Function1<OptiqConnection, T> fn)
-      throws Exception {
+        throws Exception {
       Connection connection = connectionFactory.createConnection();
       try {
         T t = fn.apply((OptiqConnection) connection);
@@ -799,7 +799,7 @@ public class OptiqAssert {
 
     /** Creates a {@link DataContext} and executes a callback. */
     public <T> AssertThat doWithDataContext(Function1<DataContext, T> fn)
-      throws Exception {
+        throws Exception {
       OptiqConnection connection = connectionFactory.createConnection();
       final DataContext dataContext = MetaImpl.createDataContext(connection);
       try {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java b/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
index 3256770..21503f2 100644
--- a/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
+++ b/core/src/test/java/net/hydromatic/optiq/tools/PlannerTest.java
@@ -369,7 +369,7 @@ public class PlannerTest {
    * a different instance causes problems unless planner state is wiped clean
    * between calls to {@link Planner#transform}. */
   @Test public void testPlanTransformWithDiffRuleSetAndConvention()
-    throws Exception {
+      throws Exception {
     Program program0 =
         Programs.ofRules(
             MergeFilterRule.INSTANCE,
@@ -401,7 +401,7 @@ public class PlannerTest {
 
   /** Unit test that plans a query with a large number of joins. */
   @Test public void testPlanNWayJoin()
-    throws Exception {
+      throws Exception {
     // Here the times before and after enabling LoptOptimizeJoinRule.
     //
     // Note the jump between N=6 and N=7; LoptOptimizeJoinRule is disabled if

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/org/eigenbase/test/DiffTestCase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/DiffTestCase.java b/core/src/test/java/org/eigenbase/test/DiffTestCase.java
index 5decf51..6df6e4f 100644
--- a/core/src/test/java/org/eigenbase/test/DiffTestCase.java
+++ b/core/src/test/java/org/eigenbase/test/DiffTestCase.java
@@ -149,7 +149,7 @@ public abstract class DiffTestCase {
    *                        extension
    */
   protected OutputStream openTestLogOutputStream(File testFileSansExt)
-    throws IOException {
+      throws IOException {
     assert logOutputStream == null;
 
     logFile = new File(testFileSansExt.toString() + ".log");

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/org/eigenbase/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/RelOptRulesTest.java b/core/src/test/java/org/eigenbase/test/RelOptRulesTest.java
index 1633024..db33f10 100644
--- a/core/src/test/java/org/eigenbase/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/eigenbase/test/RelOptRulesTest.java
@@ -794,22 +794,22 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   @Test public void testPullConstantThroughAggregatePermutedConstGroupBy()
-    throws Exception {
+      throws Exception {
     basePullConstantTroughAggregate();
   }
 
   @Test public void testPullConstantThroughAggregateConstGroupBy()
-    throws Exception {
+      throws Exception {
     basePullConstantTroughAggregate();
   }
 
   @Test public void testPullConstantThroughAggregateAllConst()
-    throws Exception {
+      throws Exception {
     basePullConstantTroughAggregate();
   }
 
   @Test public void testPullConstantThroughAggregateAllLiterals()
-    throws Exception {
+      throws Exception {
     basePullConstantTroughAggregate();
   }
 
@@ -898,7 +898,7 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   @Test public void testTransitiveInferencePreventProjectPullup()
-    throws Exception {
+      throws Exception {
     transitiveInference();
   }
 
@@ -924,7 +924,7 @@ public class RelOptRulesTest extends RelOptTestBase {
   }
 
   @Test public void testTransitiveInferenceConstantEquiPredicate()
-    throws Exception {
+      throws Exception {
     transitiveInference();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandGenerator.java b/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandGenerator.java
index e3e151a..99dfc09 100644
--- a/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandGenerator.java
+++ b/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandGenerator.java
@@ -436,7 +436,7 @@ public class ConcurrentTestCommandGenerator {
   }
 
   protected void postExecute(ConcurrentTestCommandExecutor[] threads)
-    throws Exception {
+      throws Exception {
     // check for failures
     if (requiresCustomErrorHandling()) {
       for (ConcurrentTestCommandExecutor executor : threads) {
@@ -708,7 +708,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws Exception {
+        throws Exception {
       executor.getSynchronizer().waitForOthers();
     }
   }
@@ -736,7 +736,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws Exception {
+        throws Exception {
       Thread.sleep(millis);
     }
   }
@@ -754,7 +754,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       Statement stmt = executor.getConnection().createStatement();
 
       try {
@@ -790,7 +790,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       PreparedStatement stmt =
           executor.getConnection().prepareStatement(sql);
 
@@ -804,7 +804,7 @@ public class ConcurrentTestCommandGenerator {
    */
   private static class CloseCommand extends AbstractCommand {
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       Statement stmt = executor.getStatement();
 
       if (stmt != null) {
@@ -852,7 +852,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       PreparedStatement stmt =
           (PreparedStatement) executor.getStatement();
 
@@ -1301,7 +1301,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       Statement stmt = executor.getConnection().createStatement();
 
       setTimeout(stmt);
@@ -1316,7 +1316,7 @@ public class ConcurrentTestCommandGenerator {
    */
   private static class CommitCommand extends AbstractCommand {
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       executor.getConnection().commit();
     }
   }
@@ -1327,7 +1327,7 @@ public class ConcurrentTestCommandGenerator {
    */
   private static class RollbackCommand extends AbstractCommand {
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       executor.getConnection().rollback();
     }
   }
@@ -1345,7 +1345,7 @@ public class ConcurrentTestCommandGenerator {
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       Statement stmt = executor.getConnection().createStatement();
 
       try {

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandScript.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandScript.java b/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandScript.java
index b6a3a5d..3c35c29 100644
--- a/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandScript.java
+++ b/core/src/test/java/org/eigenbase/test/concurrent/ConcurrentTestCommandScript.java
@@ -255,7 +255,7 @@ public class ConcurrentTestCommandScript
    * BINDINGS
    */
   private void prepare(String filename, List<String> bindings)
-    throws IOException {
+      throws IOException {
     vars = new VariableTable();
     CommandParser parser = new CommandParser();
     parser.rememberVariableRebindings(bindings);
@@ -334,7 +334,7 @@ public class ConcurrentTestCommandScript
   }
 
   protected void executeCommands(int threadID, List<String> commands)
-    throws Exception {
+      throws Exception {
     if (commands == null || commands.size() == 0) {
       return;
     }
@@ -416,7 +416,7 @@ public class ConcurrentTestCommandScript
 
   // timeout < 0 means no timeout
   private void storeResults(Integer threadId, ResultSet rset, long timeout)
-    throws SQLException {
+      throws SQLException {
     ResultsReader r = threadResultsReaders.get(threadId);
     r.read(rset, timeout);
   }
@@ -531,7 +531,7 @@ public class ConcurrentTestCommandScript
   }
 
   private void printThreadResults(BufferedWriter out, String[] threadResult)
-    throws IOException {
+      throws IOException {
     if (threadResult == null) {
       return;
     }
@@ -1311,7 +1311,7 @@ public class ConcurrentTestCommandScript
      * <code>in</code> until a line ending with a semicolon is found.
      */
     private String readSql(String startOfSql, BufferedReader in)
-      throws IOException {
+        throws IOException {
       // REVIEW mb StringBuffer not always needed
       StringBuffer sql = new StringBuffer(startOfSql);
       sql.append('\n');
@@ -1390,7 +1390,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       Integer threadId = executor.getThreadId();
       BufferedWriter out = threadBufferedWriters.get(threadId);
       threadResultsReaders.put(
@@ -1406,7 +1406,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       storeMessage(executor.getThreadId(), msg);
     }
   }
@@ -1423,7 +1423,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(final ConcurrentTestCommandExecutor exec)
-      throws Exception {
+        throws Exception {
       ConcurrentTestPluginCommand.TestContext context =
           new ConcurrentTestPluginCommand.TestContext() {
             public void storeMessage(String message) {
@@ -1576,7 +1576,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       // TODO: trim and chop in constructor; stash sql in base class;
       // execute() calls storeSql.
       String properSql = sql.trim();
@@ -1631,7 +1631,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       String properSql = sql.trim();
 
       storeSql(
@@ -1708,7 +1708,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       String properSql = sql.trim();
 
       storeSql(
@@ -1737,7 +1737,7 @@ public class ConcurrentTestCommandScript
     }
 
     protected void doExecute(ConcurrentTestCommandExecutor executor)
-      throws SQLException {
+        throws SQLException {
       PreparedStatement stmt =
           (PreparedStatement) executor.getStatement();
       long timeout = setTimeout(stmt);

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoAggregateRel.java
----------------------------------------------------------------------
diff --git a/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoAggregateRel.java b/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoAggregateRel.java
index 9144e39..85201ef 100644
--- a/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoAggregateRel.java
+++ b/mongodb/src/main/java/net/hydromatic/optiq/impl/mongodb/MongoAggregateRel.java
@@ -40,7 +40,7 @@ public class MongoAggregateRel
       RelNode child,
       BitSet groupSet,
       List<AggregateCall> aggCalls)
-    throws InvalidRelException {
+      throws InvalidRelException {
     super(cluster, traitSet, child, groupSet, aggCalls);
     assert getConvention() == MongoRel.CONVENTION;
     assert getConvention() == child.getConvention();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index bdc10ad..b9e3101 100644
--- a/pom.xml
+++ b/pom.xml
@@ -261,7 +261,7 @@ limitations under the License.
       <plugin>
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-checkstyle-plugin</artifactId>
-        <version>2.10</version>
+        <version>2.12.1</version>
         <executions>
           <execution>
             <id>validate</id>

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/search/SplunkConnectionImpl.java
----------------------------------------------------------------------
diff --git a/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/search/SplunkConnectionImpl.java b/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/search/SplunkConnectionImpl.java
index 7d86646..36475ba 100644
--- a/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/search/SplunkConnectionImpl.java
+++ b/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/search/SplunkConnectionImpl.java
@@ -52,7 +52,7 @@ public class SplunkConnectionImpl implements SplunkConnection {
   final Map<String, String> requestHeaders = new HashMap<String, String>();
 
   public SplunkConnectionImpl(String url, String username, String password)
-    throws MalformedURLException {
+      throws MalformedURLException {
     this(new URL(url), username, password);
   }
 
@@ -172,7 +172,7 @@ public class SplunkConnectionImpl implements SplunkConnection {
   }
 
   private static void parseResults(InputStream in, SearchResultListener srl)
-    throws IOException {
+      throws IOException {
     CSVReader csvr = new CSVReader(new InputStreamReader(in));
     try {
       String [] header = csvr.readNext();

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/util/HttpUtils.java
----------------------------------------------------------------------
diff --git a/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/util/HttpUtils.java b/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/util/HttpUtils.java
index fc0923b..72802a3 100644
--- a/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/util/HttpUtils.java
+++ b/splunk/src/main/java/net/hydromatic/optiq/impl/splunk/util/HttpUtils.java
@@ -34,7 +34,7 @@ public class HttpUtils {
       StringUtils.getClassTracer(HttpUtils.class);
 
   public static HttpURLConnection getURLConnection(String url)
-    throws IOException {
+      throws IOException {
     URLConnection conn = new URL(url).openConnection();
     final HttpURLConnection httpConn = (HttpURLConnection) conn;
 

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/splunk/src/test/java/net/hydromatic/optiq/test/SplunkAdapterTest.java
----------------------------------------------------------------------
diff --git a/splunk/src/test/java/net/hydromatic/optiq/test/SplunkAdapterTest.java b/splunk/src/test/java/net/hydromatic/optiq/test/SplunkAdapterTest.java
index 8ead06d..2254fbc 100644
--- a/splunk/src/test/java/net/hydromatic/optiq/test/SplunkAdapterTest.java
+++ b/splunk/src/test/java/net/hydromatic/optiq/test/SplunkAdapterTest.java
@@ -273,7 +273,7 @@ public class SplunkAdapterTest {
   }
 
   private void checkSql(String sql, Function<ResultSet, Void> f)
-    throws SQLException {
+      throws SQLException {
     if (!enabled()) {
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/src/main/config/checkstyle/suppressions.xml
----------------------------------------------------------------------
diff --git a/src/main/config/checkstyle/suppressions.xml b/src/main/config/checkstyle/suppressions.xml
index 4f9d5a0..3afcf5b 100644
--- a/src/main/config/checkstyle/suppressions.xml
+++ b/src/main/config/checkstyle/suppressions.xml
@@ -25,6 +25,9 @@ limitations under the License.
   <suppress checks=".*" files="EigenbaseResource.properties"/>
   <suppress checks=".*" files="net-hydromatic-optiq-jdbc.properties"/>
   <suppress checks=".*" files="Foo.java"/>
+  <suppress checks=".*" files=".*/target/maven-archiver/pom.properties"/>
+  <suppress checks=".*" files="git.properties"/>
+  <suppress checks=".*" files="trace.properties"/>
 
   <!-- This file triggers https://github.com/checkstyle/checkstyle/issues/92,
        through no fault of its own. -->

http://git-wip-us.apache.org/repos/asf/incubator-optiq/blob/cb2ad3ac/ubenchmark/src/main/java/net/hydromatic/optiq/StatementTest.java
----------------------------------------------------------------------
diff --git a/ubenchmark/src/main/java/net/hydromatic/optiq/StatementTest.java b/ubenchmark/src/main/java/net/hydromatic/optiq/StatementTest.java
index 22dfd0d..67a9d60 100644
--- a/ubenchmark/src/main/java/net/hydromatic/optiq/StatementTest.java
+++ b/ubenchmark/src/main/java/net/hydromatic/optiq/StatementTest.java
@@ -113,7 +113,7 @@ public class StatementTest {
 
   @GenerateMicroBenchmark
   public String bindExecute(HrPreparedStatement state)
-    throws SQLException {
+      throws SQLException {
     PreparedStatement st = state.ps;
     ResultSet rs = null;
     String ename = null;