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 2018/02/16 21:51:14 UTC

[3/7] calcite git commit: [CALCITE-2128] Add SQL dialect for Jethro Data (Jonathan Doron)

[CALCITE-2128] Add SQL dialect for Jethro Data (Jonathan Doron)

Before creating a JethroDataSqlDialect, SqlDialectFactoryImpl issues a
"show functions extended" command to Jethro, storing the results in a
cache for next time, and passes the list of supported functions to the
dialect.

Close apache/calcite#602


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

Branch: refs/heads/master
Commit: facd83d311098dd3f35294ba969a19cbf6d54467
Parents: 0ced3b7
Author: msydoron <ms...@gmail.com>
Authored: Mon Dec 18 15:30:55 2017 +0200
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Feb 16 10:18:01 2018 -0800

----------------------------------------------------------------------
 .../apache/calcite/adapter/jdbc/JdbcRules.java  | 148 +++++++-----
 .../apache/calcite/adapter/jdbc/JdbcSchema.java |   5 +-
 .../apache/calcite/adapter/jdbc/JdbcTable.java  |   2 +-
 .../calcite/adapter/jdbc/JdbcTableScan.java     |   2 +-
 .../rules/AbstractJoinExtractFilterRule.java    |  85 +++++++
 .../rel/rules/JoinExtractFilterRule.java        |  49 +---
 .../java/org/apache/calcite/sql/SqlDialect.java |  43 +++-
 .../apache/calcite/sql/SqlDialectFactory.java   |   4 +-
 .../calcite/sql/SqlDialectFactoryImpl.java      |  21 ++
 .../sql/dialect/JethroDataSqlDialect.java       | 235 +++++++++++++++++++
 .../rel/rel2sql/RelToSqlConverterTest.java      |  24 ++
 11 files changed, 490 insertions(+), 128 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/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 86b44af..44f9c09 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
@@ -35,6 +35,7 @@ import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.core.Aggregate;
 import org.apache.calcite.rel.core.AggregateCall;
+import org.apache.calcite.rel.core.Calc;
 import org.apache.calcite.rel.core.CorrelationId;
 import org.apache.calcite.rel.core.Filter;
 import org.apache.calcite.rel.core.Intersect;
@@ -47,16 +48,6 @@ import org.apache.calcite.rel.core.Sort;
 import org.apache.calcite.rel.core.TableModify;
 import org.apache.calcite.rel.core.Union;
 import org.apache.calcite.rel.core.Values;
-import org.apache.calcite.rel.logical.LogicalAggregate;
-import org.apache.calcite.rel.logical.LogicalCalc;
-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.LogicalProject;
-import org.apache.calcite.rel.logical.LogicalTableModify;
-import org.apache.calcite.rel.logical.LogicalUnion;
-import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.RelMdUtil;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.rel2sql.SqlImplementor;
@@ -121,23 +112,35 @@ public class JdbcRules {
   }
 
   /** Rule that converts a join to JDBC. */
-  private static class JdbcJoinRule extends JdbcConverterRule {
-    private JdbcJoinRule(JdbcConvention out) {
-      super(LogicalJoin.class, Convention.NONE, out, "JdbcJoinRule");
+  public static class JdbcJoinRule extends JdbcConverterRule {
+    /** Creates a JdbcJoinRule. */
+    public JdbcJoinRule(JdbcConvention out) {
+      super(Join.class, Convention.NONE, out, "JdbcJoinRule");
     }
 
     @Override public RelNode convert(RelNode rel) {
-      LogicalJoin join = (LogicalJoin) rel;
+      return convert((Join) rel, true);
+    }
+
+    /**
+     * Converts a {@code Join} into a {@code JdbcJoin}.
+     *
+     * @param join Join operator to convert
+     * @param convertInputTraits Whether to convert input to {@code join}'s
+     *                            JDBC convention
+     * @return A new JdbcJoin
+     */
+    public RelNode convert(Join join, boolean convertInputTraits) {
       final List<RelNode> newInputs = new ArrayList<>();
       for (RelNode input : join.getInputs()) {
-        if (!(input.getConvention() == getOutTrait())) {
+        if (convertInputTraits && input.getConvention() != getOutTrait()) {
           input =
               convert(input,
                   input.getTraitSet().replace(out));
         }
         newInputs.add(input);
       }
-      if (!canJoinOnCondition(join.getCondition())) {
+      if (convertInputTraits && !canJoinOnCondition(join.getCondition())) {
         return null;
       }
       try {
@@ -200,7 +203,7 @@ public class JdbcRules {
   /** Join operator implemented in JDBC convention. */
   public static class JdbcJoin extends Join implements JdbcRel {
     /** Creates a JdbcJoin. */
-    protected JdbcJoin(RelOptCluster cluster, RelTraitSet traitSet,
+    public JdbcJoin(RelOptCluster cluster, RelTraitSet traitSet,
         RelNode left, RelNode right, RexNode condition,
         Set<CorrelationId> variablesSet, JoinRelType joinType)
         throws InvalidRelException {
@@ -254,16 +257,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalCalc} to an
+   * Rule to convert a {@link org.apache.calcite.rel.core.Calc} to an
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcCalc}.
    */
   private static class JdbcCalcRule extends JdbcConverterRule {
     private JdbcCalcRule(JdbcConvention out) {
-      super(LogicalCalc.class, Convention.NONE, out, "JdbcCalcRule");
+      super(Calc.class, Convention.NONE, out, "JdbcCalcRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalCalc calc = (LogicalCalc) rel;
+      final Calc calc = (Calc) rel;
 
       // If there's a multiset, let FarragoMultisetSplitter work on it
       // first.
@@ -327,16 +330,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalProject} to
+   * Rule to convert a {@link org.apache.calcite.rel.core.Project} to
    * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcProject}.
    */
-  private static class JdbcProjectRule extends JdbcConverterRule {
-    private JdbcProjectRule(JdbcConvention out) {
-      super(LogicalProject.class, Convention.NONE, out, "JdbcProjectRule");
+  public static class JdbcProjectRule extends JdbcConverterRule {
+    public JdbcProjectRule(JdbcConvention out) {
+      super(Project.class, Convention.NONE, out, "JdbcProjectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalProject project = (LogicalProject) rel;
+      final Project project = (Project) rel;
 
       return new JdbcProject(
           rel.getCluster(),
@@ -349,7 +352,7 @@ public class JdbcRules {
     }
   }
 
-  /** Implementation of {@link org.apache.calcite.rel.logical.LogicalProject} in
+  /** Implementation of {@link org.apache.calcite.rel.core.Project} in
    * {@link JdbcConvention jdbc calling convention}. */
   public static class JdbcProject
       extends Project
@@ -388,16 +391,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalFilter} to
+   * Rule to convert a {@link org.apache.calcite.rel.core.Filter} to
    * an {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcFilter}.
    */
-  private static class JdbcFilterRule extends JdbcConverterRule {
-    private JdbcFilterRule(JdbcConvention out) {
-      super(LogicalFilter.class, Convention.NONE, out, "JdbcFilterRule");
+  public static class JdbcFilterRule extends JdbcConverterRule {
+    public JdbcFilterRule(JdbcConvention out) {
+      super(Filter.class, Convention.NONE, out, "JdbcFilterRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalFilter filter = (LogicalFilter) rel;
+      final Filter filter = (Filter) rel;
 
       return new JdbcFilter(
           rel.getCluster(),
@@ -431,16 +434,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalAggregate}
+   * Rule to convert a {@link org.apache.calcite.rel.core.Aggregate}
    * to a {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcAggregate}.
    */
-  private static class JdbcAggregateRule extends JdbcConverterRule {
-    private JdbcAggregateRule(JdbcConvention out) {
-      super(LogicalAggregate.class, Convention.NONE, out, "JdbcAggregateRule");
+  public static class JdbcAggregateRule extends JdbcConverterRule {
+    public JdbcAggregateRule(JdbcConvention out) {
+      super(Aggregate.class, Convention.NONE, out, "JdbcAggregateRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalAggregate agg = (LogicalAggregate) rel;
+      final Aggregate agg = (Aggregate) rel;
       if (agg.getGroupSets().size() != 1) {
         // GROUPING SETS not supported; see
         // [CALCITE-734] Push GROUPING SETS to underlying SQL via JDBC adapter
@@ -511,17 +514,36 @@ public class JdbcRules {
    * Rule to convert a {@link org.apache.calcite.rel.core.Sort} to an
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcSort}.
    */
-  private static class JdbcSortRule extends JdbcConverterRule {
-    private JdbcSortRule(JdbcConvention out) {
+  public static class JdbcSortRule extends JdbcConverterRule {
+    /** Creates a JdbcSortRule. */
+    public JdbcSortRule(JdbcConvention out) {
       super(Sort.class, Convention.NONE, out, "JdbcSortRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final Sort sort = (Sort) rel;
+      return convert((Sort) rel, true);
+    }
+
+    /**
+     * Converts a {@code Sort} into a {@code JdbcSort}.
+     *
+     * @param sort Sort operator to convert
+     * @param convertInputTraits Whether to convert input to {@code sort}'s
+     *                            JDBC convention
+     * @return A new JdbcSort
+     */
+    public RelNode convert(Sort sort, boolean convertInputTraits) {
       final RelTraitSet traitSet = sort.getTraitSet().replace(out);
-      return new JdbcSort(rel.getCluster(), traitSet,
-          convert(sort.getInput(), traitSet), sort.getCollation(), sort.offset,
-          sort.fetch);
+
+      final RelNode input;
+      if (convertInputTraits) {
+        input = convert(sort.getInput(), traitSet);
+      } else {
+        input = sort.getInput();
+      }
+
+      return new JdbcSort(sort.getCluster(), traitSet,
+          input, sort.getCollation(), sort.offset, sort.fetch);
     }
   }
 
@@ -553,16 +575,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert an {@link org.apache.calcite.rel.logical.LogicalUnion} to a
+   * Rule to convert an {@link org.apache.calcite.rel.core.Union} to a
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcUnion}.
    */
-  private static class JdbcUnionRule extends JdbcConverterRule {
-    private JdbcUnionRule(JdbcConvention out) {
-      super(LogicalUnion.class, Convention.NONE, out, "JdbcUnionRule");
+  public static class JdbcUnionRule extends JdbcConverterRule {
+    public JdbcUnionRule(JdbcConvention out) {
+      super(Union.class, Convention.NONE, out, "JdbcUnionRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalUnion union = (LogicalUnion) rel;
+      final Union union = (Union) rel;
       final RelTraitSet traitSet =
           union.getTraitSet().replace(out);
       return new JdbcUnion(rel.getCluster(), traitSet,
@@ -596,16 +618,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalIntersect}
+   * Rule to convert a {@link org.apache.calcite.rel.core.Intersect}
    * to a {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcIntersect}.
    */
-  private static class JdbcIntersectRule extends JdbcConverterRule {
+  public static class JdbcIntersectRule extends JdbcConverterRule {
     private JdbcIntersectRule(JdbcConvention out) {
-      super(LogicalIntersect.class, Convention.NONE, out, "JdbcIntersectRule");
+      super(Intersect.class, Convention.NONE, out, "JdbcIntersectRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalIntersect intersect = (LogicalIntersect) rel;
+      final Intersect intersect = (Intersect) rel;
       if (intersect.all) {
         return null; // INTERSECT ALL not implemented
       }
@@ -640,16 +662,16 @@ public class JdbcRules {
   }
 
   /**
-   * Rule to convert a {@link org.apache.calcite.rel.logical.LogicalMinus} to a
+   * Rule to convert a {@link org.apache.calcite.rel.core.Minus} to a
    * {@link org.apache.calcite.adapter.jdbc.JdbcRules.JdbcMinus}.
    */
-  private static class JdbcMinusRule extends JdbcConverterRule {
+  public static class JdbcMinusRule extends JdbcConverterRule {
     private JdbcMinusRule(JdbcConvention out) {
-      super(LogicalMinus.class, Convention.NONE, out, "JdbcMinusRule");
+      super(Minus.class, Convention.NONE, out, "JdbcMinusRule");
     }
 
     public RelNode convert(RelNode rel) {
-      final LogicalMinus minus = (LogicalMinus) rel;
+      final Minus minus = (Minus) rel;
       if (minus.all) {
         return null; // EXCEPT ALL not implemented
       }
@@ -681,20 +703,16 @@ public class JdbcRules {
   /** Rule that converts a table-modification to JDBC. */
   public static class JdbcTableModificationRule extends JdbcConverterRule {
     private JdbcTableModificationRule(JdbcConvention out) {
-      super(
-          LogicalTableModify.class,
-          Convention.NONE,
-          out,
+      super(TableModify.class, Convention.NONE, out,
           "JdbcTableModificationRule");
     }
 
     @Override public RelNode convert(RelNode rel) {
-      final LogicalTableModify modify =
-          (LogicalTableModify) rel;
+      final TableModify modify =
+          (TableModify) rel;
       final ModifiableTable modifiableTable =
           modify.getTable().unwrap(ModifiableTable.class);
-      if (modifiableTable == null
-          /* || modifiableTable.getExpression(tableInSchema) == null */) {
+      if (modifiableTable == null) {
         return null;
       }
       final RelTraitSet traitSet =
@@ -759,11 +777,11 @@ public class JdbcRules {
   /** Rule that converts a values operator to JDBC. */
   public static class JdbcValuesRule extends JdbcConverterRule {
     private JdbcValuesRule(JdbcConvention out) {
-      super(LogicalValues.class, Convention.NONE, out, "JdbcValuesRule");
+      super(Values.class, Convention.NONE, out, "JdbcValuesRule");
     }
 
     @Override public RelNode convert(RelNode rel) {
-      LogicalValues values = (LogicalValues) rel;
+      Values values = (Values) rel;
       return new JdbcValues(values.getCluster(), values.getRowType(),
           values.getTuples(), values.getTraitSet().replace(out));
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
index 8e7ecdf..eb91b14 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcSchema.java
@@ -104,7 +104,8 @@ public class JdbcSchema implements Schema {
       DataSource dataSource,
       String catalog,
       String schema) {
-    return create(parentSchema, name, dataSource, new SqlDialectFactoryImpl(), catalog, schema);
+    return create(parentSchema, name, dataSource,
+        SqlDialectFactoryImpl.INSTANCE, catalog, schema);
   }
 
   public static JdbcSchema create(
@@ -174,7 +175,7 @@ public class JdbcSchema implements Schema {
    */
   @Deprecated // to be removed before 2.0
   public static SqlDialect createDialect(DataSource dataSource) {
-    return createDialect(new SqlDialectFactoryImpl(), dataSource);
+    return createDialect(SqlDialectFactoryImpl.INSTANCE, dataSource);
   }
 
   /** Returns a suitable SQL dialect for the given data source. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
index b093eb2..02b6207 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTable.java
@@ -74,7 +74,7 @@ import java.util.List;
  * The resulting queryable can then be converted to a SQL query, which can be
  * executed efficiently on the JDBC server.</p>
  */
-class JdbcTable extends AbstractQueryableTable
+public class JdbcTable extends AbstractQueryableTable
     implements TranslatableTable, ScannableTable, ModifiableTable {
   private RelProtoDataType protoRowType;
   private final JdbcSchema jdbcSchema;

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
index 7ef8938..b621b85 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcTableScan.java
@@ -30,7 +30,7 @@ import java.util.List;
  * Relational expression representing a scan of a table in a JDBC data source.
  */
 public class JdbcTableScan extends TableScan implements JdbcRel {
-  final JdbcTable jdbcTable;
+  protected final JdbcTable jdbcTable;
 
   protected JdbcTableScan(
       RelOptCluster cluster,

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
new file mode 100644
index 0000000..106aa28
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/rel/rules/AbstractJoinExtractFilterRule.java
@@ -0,0 +1,85 @@
+/*
+ * 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.RelOptRuleOperand;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.Join;
+import org.apache.calcite.rel.core.JoinRelType;
+import org.apache.calcite.tools.RelBuilder;
+import org.apache.calcite.tools.RelBuilderFactory;
+
+/**
+ * Rule to convert an
+ * {@link org.apache.calcite.rel.core.Join inner join} to a
+ * {@link org.apache.calcite.rel.core.Filter filter} on top of a
+ * {@link org.apache.calcite.rel.core.Join cartesian inner join}.
+ *
+ * <p>One benefit of this transformation is that after it, the join condition
+ * can be combined with conditions and expressions above the join. It also makes
+ * the <code>FennelCartesianJoinRule</code> applicable.
+ *
+ * <p>The constructor is parameterized to allow any sub-class of
+ * {@link org.apache.calcite.rel.core.Join}.</p>
+ */
+public abstract class AbstractJoinExtractFilterRule extends RelOptRule {
+  /** Creates an AbstractJoinExtractFilterRule. */
+  protected AbstractJoinExtractFilterRule(RelOptRuleOperand operand,
+      RelBuilderFactory relBuilderFactory, String description) {
+    super(operand, relBuilderFactory, description);
+  }
+
+  public void onMatch(RelOptRuleCall call) {
+    final Join join = call.rel(0);
+
+    if (join.getJoinType() != JoinRelType.INNER) {
+      return;
+    }
+
+    if (join.getCondition().isAlwaysTrue()) {
+      return;
+    }
+
+    if (!join.getSystemFieldList().isEmpty()) {
+      // FIXME Enable this rule for joins with system fields
+      return;
+    }
+
+    final RelBuilder builder = call.builder();
+
+    // NOTE jvs 14-Mar-2006:  See JoinCommuteRule for why we
+    // preserve attribute semiJoinDone here.
+
+    final RelNode cartesianJoin =
+        join.copy(
+            join.getTraitSet(),
+            builder.literal(true),
+            join.getLeft(),
+            join.getRight(),
+            join.getJoinType(),
+            join.isSemiJoinDone());
+
+    builder.push(cartesianJoin)
+        .filter(join.getCondition());
+
+    call.transformTo(builder.build());
+  }
+}
+
+// End AbstractJoinExtractFilterRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
index 8c4a02d..1e7e661 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinExtractFilterRule.java
@@ -16,11 +16,7 @@
  */
 package org.apache.calcite.rel.rules;
 
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.core.Join;
-import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.logical.LogicalJoin;
 import org.apache.calcite.tools.RelBuilderFactory;
@@ -39,7 +35,7 @@ import org.apache.calcite.tools.RelBuilderFactory;
  * {@link org.apache.calcite.rel.core.Join}, not just
  * {@link org.apache.calcite.rel.logical.LogicalJoin}.</p>
  */
-public final class JoinExtractFilterRule extends RelOptRule {
+public final class JoinExtractFilterRule extends AbstractJoinExtractFilterRule {
   //~ Static fields/initializers ---------------------------------------------
 
   /** The singleton. */
@@ -50,14 +46,6 @@ public final class JoinExtractFilterRule extends RelOptRule {
   //~ Constructors -----------------------------------------------------------
 
   /**
-   * Creates an JoinExtractFilterRule.
-   */
-  @Deprecated // to be removed before 2.0
-  public JoinExtractFilterRule(Class<? extends Join> clazz) {
-    this(clazz, RelFactories.LOGICAL_BUILDER);
-  }
-
-  /**
    * Creates a JoinExtractFilterRule.
    */
   public JoinExtractFilterRule(Class<? extends Join> clazz,
@@ -67,41 +55,6 @@ public final class JoinExtractFilterRule extends RelOptRule {
 
   //~ Methods ----------------------------------------------------------------
 
-  public void onMatch(RelOptRuleCall call) {
-    final Join join = call.rel(0);
-
-    if (join.getJoinType() != JoinRelType.INNER) {
-      return;
-    }
-
-    if (join.getCondition().isAlwaysTrue()) {
-      return;
-    }
-
-    if (!join.getSystemFieldList().isEmpty()) {
-      // FIXME Enable this rule for joins with system fields
-      return;
-    }
-
-    // NOTE jvs 14-Mar-2006:  See JoinCommuteRule for why we
-    // preserve attribute semiJoinDone here.
-
-    RelNode cartesianJoinRel =
-        join.copy(
-            join.getTraitSet(),
-            join.getCluster().getRexBuilder().makeLiteral(true),
-            join.getLeft(),
-            join.getRight(),
-            join.getJoinType(),
-            join.isSemiJoinDone());
-
-    final RelFactories.FilterFactory factory =
-        RelFactories.DEFAULT_FILTER_FACTORY;
-    RelNode filterRel =
-        factory.createFilter(cartesianJoinRel, join.getCondition());
-
-    call.transformTo(filterRel);
-  }
 }
 
 // End JoinExtractFilterRule.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
index 42cf7b1..07bc60a 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialect.java
@@ -23,6 +23,7 @@ import org.apache.calcite.rel.RelFieldCollation;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
+import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.BasicSqlType;
@@ -157,7 +158,7 @@ public class SqlDialect {
   /** Creates an empty context. Use {@link #EMPTY_CONTEXT} if possible. */
   protected static Context emptyContext() {
     return new ContextImpl(DatabaseProduct.UNKNOWN, null, null, -1, -1, null,
-        NullCollation.HIGH);
+        NullCollation.HIGH, JethroDataSqlDialect.JethroInfo.EMPTY);
   }
 
   /**
@@ -502,6 +503,12 @@ public class SqlDialect {
     return false;
   }
 
+  /** Returns whether this dialect supports a given function or operator. */
+  public boolean supportsFunction(SqlOperator operator, RelDataType type,
+      List<RelDataType> paramTypes) {
+    return true;
+  }
+
   public CalendarPolicy getCalendarPolicy() {
     return CalendarPolicy.NULL;
   }
@@ -778,6 +785,7 @@ public class SqlDialect {
     HIVE("Apache Hive", null, NullCollation.LOW),
     INFORMIX("Informix", null, NullCollation.HIGH),
     INGRES("Ingres", null, NullCollation.HIGH),
+    JETHRO("JethroData", "\"", NullCollation.LOW),
     LUCIDDB("LucidDB", "\"", NullCollation.HIGH),
     INTERBASE("Interbase", null, NullCollation.HIGH),
     PHOENIX("Phoenix", "\"", NullCollation.HIGH),
@@ -839,7 +847,7 @@ public class SqlDialect {
      * <p>Since databases have many versions and flavors, this dummy dialect
      * is at best an approximation. If you want exact information, better to
      * use a dialect created from an actual connection's metadata
-     * (see {@link SqlDialect#create(java.sql.DatabaseMetaData)}).
+     * (see {@link SqlDialectFactory#create(java.sql.DatabaseMetaData)}).
      *
      * @return Dialect representing lowest-common-denominator behavior for
      * all versions of this database
@@ -868,6 +876,8 @@ public class SqlDialect {
     Context withIdentifierQuoteString(String identifierQuoteString);
     @Nonnull NullCollation nullCollation();
     Context withNullCollation(@Nonnull NullCollation nullCollation);
+    JethroDataSqlDialect.JethroInfo jethroInfo();
+    Context withJethroInfo(JethroDataSqlDialect.JethroInfo jethroInfo);
   }
 
   /** Implementation of Context. */
@@ -879,11 +889,13 @@ public class SqlDialect {
     private final int databaseMinorVersion;
     private final String identifierQuoteString;
     private final NullCollation nullCollation;
+    private final JethroDataSqlDialect.JethroInfo jethroInfo;
 
     private ContextImpl(DatabaseProduct databaseProduct,
         String databaseProductName, String databaseVersion,
         int databaseMajorVersion, int databaseMinorVersion,
-        String identifierQuoteString, NullCollation nullCollation) {
+        String identifierQuoteString, NullCollation nullCollation,
+        JethroDataSqlDialect.JethroInfo jethroInfo) {
       this.databaseProduct = Preconditions.checkNotNull(databaseProduct);
       this.databaseProductName = databaseProductName;
       this.databaseVersion = databaseVersion;
@@ -891,6 +903,7 @@ public class SqlDialect {
       this.databaseMinorVersion = databaseMinorVersion;
       this.identifierQuoteString = identifierQuoteString;
       this.nullCollation = Preconditions.checkNotNull(nullCollation);
+      this.jethroInfo = Preconditions.checkNotNull(jethroInfo);
     }
 
     @Nonnull public DatabaseProduct databaseProduct() {
@@ -901,7 +914,7 @@ public class SqlDialect {
         @Nonnull DatabaseProduct databaseProduct) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     public String databaseProductName() {
@@ -911,7 +924,7 @@ public class SqlDialect {
     public Context withDatabaseProductName(String databaseProductName) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     public String databaseVersion() {
@@ -921,7 +934,7 @@ public class SqlDialect {
     public Context withDatabaseVersion(String databaseVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     public int databaseMajorVersion() {
@@ -931,7 +944,7 @@ public class SqlDialect {
     public Context withDatabaseMajorVersion(int databaseMajorVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     public int databaseMinorVersion() {
@@ -941,7 +954,7 @@ public class SqlDialect {
     public Context withDatabaseMinorVersion(int databaseMinorVersion) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     public String identifierQuoteString() {
@@ -951,7 +964,7 @@ public class SqlDialect {
     public Context withIdentifierQuoteString(String identifierQuoteString) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
     }
 
     @Nonnull public NullCollation nullCollation() {
@@ -961,7 +974,17 @@ public class SqlDialect {
     public Context withNullCollation(@Nonnull NullCollation nullCollation) {
       return new ContextImpl(databaseProduct, databaseProductName,
           databaseVersion, databaseMajorVersion, databaseMinorVersion,
-          identifierQuoteString, nullCollation);
+          identifierQuoteString, nullCollation, jethroInfo);
+    }
+
+    @Nonnull public JethroDataSqlDialect.JethroInfo jethroInfo() {
+      return jethroInfo;
+    }
+
+    public Context withJethroInfo(JethroDataSqlDialect.JethroInfo jethroInfo) {
+      return new ContextImpl(databaseProduct, databaseProductName,
+          databaseVersion, databaseMajorVersion, databaseMinorVersion,
+          identifierQuoteString, nullCollation, jethroInfo);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
index 73de073..202b8fb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactory.java
@@ -20,7 +20,7 @@ import java.sql.Connection;
 import java.sql.DatabaseMetaData;
 
 /**
- * <code>SqlDialectFactory</code> constructs a <code>SqlDialect</code> appropriate
+ * Creates a <code>SqlDialect</code> appropriate
  * for a given database metadata object.
  */
 public interface SqlDialectFactory {
@@ -34,6 +34,8 @@ public interface SqlDialectFactory {
    *
    * @param databaseMetaData used to determine which dialect of SQL to
    *                         generate
+   *
+   * @throws RuntimeException if there was an error creating the dialect
    */
   SqlDialect create(DatabaseMetaData databaseMetaData);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
index c2bf4e3..18de463 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDialectFactoryImpl.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.sql;
 
 import org.apache.calcite.config.NullCollation;
+
 import org.apache.calcite.sql.dialect.AccessSqlDialect;
 import org.apache.calcite.sql.dialect.AnsiSqlDialect;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
@@ -30,6 +31,7 @@ import org.apache.calcite.sql.dialect.InfobrightSqlDialect;
 import org.apache.calcite.sql.dialect.InformixSqlDialect;
 import org.apache.calcite.sql.dialect.IngresSqlDialect;
 import org.apache.calcite.sql.dialect.InterbaseSqlDialect;
+import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
 import org.apache.calcite.sql.dialect.LucidDbSqlDialect;
 import org.apache.calcite.sql.dialect.MssqlSqlDialect;
 import org.apache.calcite.sql.dialect.MysqlSqlDialect;
@@ -44,6 +46,9 @@ import org.apache.calcite.sql.dialect.SybaseSqlDialect;
 import org.apache.calcite.sql.dialect.TeradataSqlDialect;
 import org.apache.calcite.sql.dialect.VerticaSqlDialect;
 
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import java.sql.DatabaseMetaData;
 import java.sql.SQLException;
 import java.util.Locale;
@@ -52,14 +57,23 @@ import java.util.Locale;
  * The default implementation of a <code>SqlDialectFactory</code>.
  */
 public class SqlDialectFactoryImpl implements SqlDialectFactory {
+  private static final Logger LOGGER = LoggerFactory.getLogger(SqlDialectFactoryImpl.class);
+
+  public static final SqlDialectFactoryImpl INSTANCE = new SqlDialectFactoryImpl();
+
+  private final JethroDataSqlDialect.JethroInfoCache jethroCache =
+      JethroDataSqlDialect.createCache();
+
   public SqlDialect create(DatabaseMetaData databaseMetaData) {
     String databaseProductName;
     int databaseMajorVersion;
     int databaseMinorVersion;
+    String databaseVersion;
     try {
       databaseProductName = databaseMetaData.getDatabaseProductName();
       databaseMajorVersion = databaseMetaData.getDatabaseMajorVersion();
       databaseMinorVersion = databaseMetaData.getDatabaseMinorVersion();
+      databaseVersion = databaseMetaData.getDatabaseProductVersion();
     } catch (SQLException e) {
       throw new RuntimeException("while detecting database product", e);
     }
@@ -71,6 +85,7 @@ public class SqlDialectFactoryImpl implements SqlDialectFactory {
         .withDatabaseProductName(databaseProductName)
         .withDatabaseMajorVersion(databaseMajorVersion)
         .withDatabaseMinorVersion(databaseMinorVersion)
+        .withDatabaseVersion(databaseVersion)
         .withIdentifierQuoteString(quoteString)
         .withNullCollation(nullCollation);
     switch (upperProductName) {
@@ -86,6 +101,9 @@ public class SqlDialectFactoryImpl implements SqlDialectFactory {
       return new IngresSqlDialect(c);
     case "INTERBASE":
       return new InterbaseSqlDialect(c);
+    case "JETHRODATA":
+      return new JethroDataSqlDialect(
+          c.withJethroInfo(jethroCache.get(databaseMetaData)));
     case "LUCIDDB":
       return new LucidDbSqlDialect(c);
     case "ORACLE":
@@ -184,6 +202,8 @@ public class SqlDialectFactoryImpl implements SqlDialectFactory {
       return IngresSqlDialect.DEFAULT;
     case INTERBASE:
       return InterbaseSqlDialect.DEFAULT;
+    case JETHRO:
+      throw new RuntimeException("Jethro does not support simple creation");
     case LUCIDDB:
       return LucidDbSqlDialect.DEFAULT;
     case MSSQL:
@@ -216,6 +236,7 @@ public class SqlDialectFactoryImpl implements SqlDialectFactory {
       return null;
     }
   }
+
 }
 
 // End SqlDialectFactoryImpl.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/main/java/org/apache/calcite/sql/dialect/JethroDataSqlDialect.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/dialect/JethroDataSqlDialect.java b/core/src/main/java/org/apache/calcite/sql/dialect/JethroDataSqlDialect.java
new file mode 100644
index 0000000..26f0148
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/dialect/JethroDataSqlDialect.java
@@ -0,0 +1,235 @@
+/*
+ * 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.sql.dialect;
+
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlNode;
+import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+import com.google.common.base.Preconditions;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.collect.LinkedHashMultimap;
+import com.google.common.collect.Multimap;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Set;
+
+
+/**
+ * A <code>SqlDialect</code> implementation for the JethroData database.
+ */
+public class JethroDataSqlDialect extends SqlDialect {
+  private final JethroInfo info;
+
+  /** Creates a JethroDataSqlDialect. */
+  public JethroDataSqlDialect(Context context) {
+    super(context);
+    this.info = context.jethroInfo();
+  }
+
+  @Override public boolean supportsCharSet() {
+    return false;
+  }
+
+  @Override public SqlNode emulateNullDirection(SqlNode node,
+      boolean nullsFirst, boolean desc) {
+    return node;
+  }
+
+  @Override public boolean supportsAggregateFunction(SqlKind kind) {
+    switch (kind) {
+    case COUNT:
+    case SUM:
+    case AVG:
+    case MIN:
+    case MAX:
+    case STDDEV_POP:
+    case STDDEV_SAMP:
+    case VAR_POP:
+    case VAR_SAMP:
+      return true;
+    default:
+      break;
+    }
+    return false;
+  }
+
+  @Override public boolean supportsFunction(SqlOperator operator,
+      RelDataType type, List<RelDataType> paramTypes) {
+    switch (operator.getKind()) {
+    case IS_NOT_NULL:
+    case IS_NULL:
+    case AND:
+    case OR:
+    case NOT:
+    case BETWEEN:
+    case CASE:
+    case CAST:
+      return true;
+    }
+    final Set<JethroSupportedFunction> functions =
+        info.supportedFunctions.get(operator.getName());
+
+    if (functions != null) {
+      for (JethroSupportedFunction f : functions) {
+        if (f.argumentsMatch(paramTypes)) {
+          return true;
+        }
+      }
+    }
+    LOGGER.debug("Unsupported function in jethro: " + operator + " with params "
+        + paramTypes);
+    return false;
+  }
+
+  @SuppressWarnings("deprecation")
+  @Override public boolean supportsOffsetFetch() {
+    return false;
+  }
+
+  @Override public boolean supportsNestedAggregations() {
+    return false;
+  }
+
+  public static JethroInfoCache createCache() {
+    return new JethroInfoCacheImpl();
+  }
+
+  /** Information about a function supported by Jethro. */
+  static class JethroSupportedFunction {
+    private final List<SqlTypeName> operandTypes;
+
+    JethroSupportedFunction(String name, String operands) {
+      Preconditions.checkNotNull(name); // not currently used
+      final ImmutableList.Builder<SqlTypeName> b = ImmutableList.builder();
+      for (String strType : operands.split(":")) {
+        b.add(parse(strType));
+      }
+      this.operandTypes = b.build();
+    }
+
+    private SqlTypeName parse(String strType) {
+      switch (strType.toLowerCase(Locale.ROOT)) {
+      case "bigint":
+      case "long":
+        return SqlTypeName.BIGINT;
+      case "integer":
+      case "int":
+        return SqlTypeName.INTEGER;
+      case "double":
+        return SqlTypeName.DOUBLE;
+      case "float":
+        return SqlTypeName.FLOAT;
+      case "string":
+        return SqlTypeName.VARCHAR;
+      case "timestamp":
+        return SqlTypeName.TIMESTAMP;
+      default:
+        return SqlTypeName.ANY;
+      }
+    }
+
+    boolean argumentsMatch(List<RelDataType> paramTypes) {
+      if (paramTypes.size() != operandTypes.size()) {
+        return false;
+      }
+      for (int i = 0; i < paramTypes.size(); i++) {
+        if (paramTypes.get(i).getSqlTypeName() != operandTypes.get(i)) {
+          return false;
+        }
+      }
+      return true;
+    }
+  }
+
+  /** Stores information about capabilities of Jethro databases. */
+  public interface JethroInfoCache {
+    JethroInfo get(DatabaseMetaData databaseMetaData);
+  }
+
+  /** Implementation of {@code JethroInfoCache}. */
+  private static class JethroInfoCacheImpl implements JethroInfoCache {
+    final Map<String, JethroInfo> map = new HashMap<>();
+
+    public JethroInfo get(final DatabaseMetaData metaData) {
+      try {
+        assert "JethroData".equals(metaData.getDatabaseProductName());
+        String productVersion = metaData.getDatabaseProductVersion();
+        synchronized (JethroInfoCacheImpl.this) {
+          JethroInfo info = map.get(productVersion);
+          if (info == null) {
+            final Connection c = metaData.getConnection();
+            info = makeInfo(c);
+            map.put(productVersion, info);
+          }
+          return info;
+        }
+      } catch (Exception e) {
+        LOGGER.error("Failed to create JethroDataDialect", e);
+        throw new RuntimeException("Failed to create JethroDataDialect", e);
+      }
+    }
+
+    private JethroInfo makeInfo(Connection jethroConnection) {
+      try (Statement jethroStatement = jethroConnection.createStatement();
+           ResultSet functionsTupleSet =
+               jethroStatement.executeQuery("show functions extended")) {
+        final Multimap<String, JethroSupportedFunction> supportedFunctions =
+            LinkedHashMultimap.create();
+        while (functionsTupleSet.next()) {
+          String functionName = functionsTupleSet.getString(1);
+          String operandsType = functionsTupleSet.getString(3);
+          supportedFunctions.put(functionName,
+              new JethroSupportedFunction(functionName, operandsType));
+        }
+        return new JethroInfo(supportedFunctions);
+      } catch (Exception e) {
+        final String msg =
+            "Jethro server failed to execute 'show functions extended'";
+        LOGGER.error(msg, e);
+        throw new RuntimeException(msg
+            + "; make sure your Jethro server is up to date", e);
+      }
+    }
+  }
+
+  /** Information about the capabilities of a Jethro database. */
+  public static class JethroInfo {
+    public static final JethroInfo EMPTY = new JethroInfo(
+        ImmutableSetMultimap.<String, JethroSupportedFunction>of());
+
+    private final ImmutableSetMultimap<String, JethroSupportedFunction> supportedFunctions;
+
+    public JethroInfo(Multimap<String, JethroSupportedFunction> supportedFunctions) {
+      this.supportedFunctions = ImmutableSetMultimap.copyOf(supportedFunctions);
+    }
+  }
+}
+
+// End JethroDataSqlDialect.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/facd83d3/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
index d6bb976..2cdda74 100644
--- a/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
+++ b/core/src/test/java/org/apache/calcite/rel/rel2sql/RelToSqlConverterTest.java
@@ -29,9 +29,11 @@ import org.apache.calcite.rel.rules.UnionMergeRule;
 import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.sql.SqlDialect;
+import org.apache.calcite.sql.SqlDialect.Context;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.dialect.CalciteSqlDialect;
 import org.apache.calcite.sql.dialect.HiveSqlDialect;
+import org.apache.calcite.sql.dialect.JethroDataSqlDialect;
 import org.apache.calcite.sql.dialect.MysqlSqlDialect;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql2rel.SqlToRelConverter;
@@ -95,6 +97,18 @@ public class RelToSqlConverterTest {
     return Frameworks.getPlanner(config);
   }
 
+  private static JethroDataSqlDialect jethroDataSqlDialect() {
+    Context dummyContext = SqlDialect.EMPTY_CONTEXT
+        .withDatabaseProduct(SqlDialect.DatabaseProduct.JETHRO)
+        .withDatabaseMajorVersion(1)
+        .withDatabaseMinorVersion(0)
+        .withDatabaseVersion("1.0")
+        .withIdentifierQuoteString("\"")
+        .withNullCollation(NullCollation.HIGH)
+        .withJethroInfo(JethroDataSqlDialect.JethroInfo.EMPTY);
+    return new JethroDataSqlDialect(dummyContext);
+  }
+
   private static MysqlSqlDialect mySqlDialect(NullCollation nullCollation) {
     return new MysqlSqlDialect(SqlDialect.EMPTY_CONTEXT
         .withDatabaseProduct(SqlDialect.DatabaseProduct.MYSQL)
@@ -422,6 +436,16 @@ public class RelToSqlConverterTest {
     sql(query).dialect(hive2_1_0_Dialect).ok(expected);
   }
 
+  @Test public void testJethroDataSelectQueryWithOrderByDescAndNullsFirstShouldBeEmulated() {
+    final String query = "select \"product_id\" from \"product\"\n"
+        + "order by \"product_id\" desc nulls first";
+
+    final String expected = "SELECT \"product_id\"\n"
+        + "FROM \"foodmart\".\"product\"\n"
+        + "ORDER BY \"product_id\", \"product_id\" DESC";
+    sql(query).dialect(jethroDataSqlDialect()).ok(expected);
+  }
+
   @Test public void testMySqlSelectQueryWithOrderByDescAndNullsFirstShouldBeEmulated() {
     final String query = "select \"product_id\" from \"product\"\n"
         + "order by \"product_id\" desc nulls first";