You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2015/09/14 07:28:48 UTC

[01/15] drill git commit: DRILL-3180: Initial JDBC plugin implementation.

Repository: drill
Updated Branches:
  refs/heads/master b525692e0 -> e43155d8e


http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
new file mode 100644
index 0000000..4685046
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
@@ -0,0 +1,119 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.physical.base.AbstractBase;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalVisitor;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.apache.drill.exec.store.ischema.SelectedTable;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterators;
+
+public class MPJdbcSubScan extends AbstractBase implements SubScan {
+  private MPJdbcFormatPlugin plugin;
+  private MPJdbcFormatConfig pluginConfig;
+  private List<SchemaPath> columns;
+  private List<MPJdbcScanSpec> scanList;
+  private String userName;
+
+  @JsonCreator
+  public MPJdbcSubScan(@JacksonInject StoragePluginRegistry registry,
+      @JsonProperty("userName") String userName,
+      @JsonProperty("pluginConfig") MPJdbcFormatConfig pluginConfig,
+      @JsonProperty("ScanList") List<MPJdbcScanSpec> scanlist,
+      @JsonProperty("columns") List<SchemaPath> columns) throws ExecutionSetupException {
+    super(userName);
+    this.plugin = (MPJdbcFormatPlugin) registry.getPlugin(pluginConfig);
+    this.pluginConfig = pluginConfig;
+    this.columns = columns;
+    this.scanList = scanlist;
+    this.userName = userName;
+  }
+  public MPJdbcSubScan(MPJdbcFormatPlugin plugin,
+          @JsonProperty("userName") String userName,
+          @JsonProperty("pluginConfig") MPJdbcFormatConfig pluginConfig,
+          @JsonProperty("ScanList") List<MPJdbcScanSpec> scanlist,
+          @JsonProperty("columns") List<SchemaPath> columns) {
+    super(userName);
+    this.plugin = plugin;
+    this.pluginConfig = pluginConfig;
+    this.columns = columns;
+    this.scanList = scanlist;
+    this.userName = userName;
+    }
+
+  @Override
+  public int getOperatorType() {
+    return 55;
+  }
+
+  @JsonProperty("columns")
+  public List<SchemaPath> getColumns() {
+    // TODO Auto-generated method stub
+    return columns;
+  }
+
+  @JsonIgnore
+  public List<MPJdbcScanSpec> getScanList() {
+    return this.scanList;
+  }
+
+  @JsonIgnore
+  public MPJdbcFormatConfig getConfig() {
+    return this.pluginConfig;
+  }
+
+  @JsonIgnore
+  public MPJdbcFormatPlugin getPlugin() {
+    return this.plugin;
+  }
+
+  @Override
+  public boolean isExecutable() {
+    return false;
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(
+      PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
+    return physicalVisitor.visitSubScan(this, value);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MPJdbcSubScan(plugin,userName, pluginConfig, scanList, columns);
+  }
+
+  @Override
+  public Iterator<PhysicalOperator> iterator() {
+    return Iterators.emptyIterator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
new file mode 100755
index 0000000..d408322
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
@@ -0,0 +1,12 @@
+{
+  "storage":{
+    jdbc : {
+      type:"jdbc",
+      enabled: false,
+      driver:"",
+      uri:"jdbc://",
+      username:"",
+      password:""
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml b/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
new file mode 100644
index 0000000..0758908
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
@@ -0,0 +1,41 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<!DOCTYPE module PUBLIC
+    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
+    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
+
+<!--  Checkstyle configuration for Apache Drill -->
+
+<module name="Checker">
+
+  <module name="TreeWalker">
+
+    <module name="IllegalImport">
+      <!-- For "org.apache.commons.lang.*" classes use the corresponding class from "org.apache.commons.lang3.*" -->
+      <property name="illegalPkgs" value="com.google.hive12,com.beust.jcommander.internal,jersey.repackaged,org.apache.commons.lang"/>
+    </module>
+
+    <module name="AvoidStarImport">
+      <property name="allowStaticMemberImports" value="true"/>
+    </module>
+
+    <module name="NeedBraces"/>
+
+  </module>
+
+  <module name="FileTabCharacter"/>
+
+  <module name="RegexpSingleline">
+    <property name="format" value="\s+$"/>
+  </module>
+
+</module>

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml b/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
new file mode 100644
index 0000000..9d4682b
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
@@ -0,0 +1,19 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!-- 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. -->
+<!DOCTYPE suppressions PUBLIC
+    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
+    "suppressions_1_1.dtd">
+
+<!--  Checkstyle Suppressions for Apache Drill -->
+<suppressions>
+  <suppress files="[\\/]generated-sources[\\/]" checks="AvoidStarImport,NeedBraces"/>
+</suppressions>

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/drill-module.conf b/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
new file mode 100644
index 0000000..e753703
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
@@ -0,0 +1,30 @@
+// 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.
+//
+//  This file tells Drill to consider this module when class path scanning.
+//  This file can also include any supplementary configuration information.
+//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
+
+drill.exec: {
+
+  sys.store.provider: {
+    jdbc : {
+      driver: "undefined",
+      uri : "jdbc://localhost",
+      username : "user",
+      password: "password"
+    }
+  }
+}


[03/15] drill git commit: DRILL-3180: JDBC Storage Plugin updates.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
index 1c36e79..ca08363 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRel.java
@@ -19,28 +19,25 @@ package org.apache.drill.exec.planner.logical;
 
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Set;
 
-import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.logical.data.Join;
-import org.apache.drill.common.logical.data.JoinCondition;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.common.logical.data.Project;
-import org.apache.drill.exec.planner.common.DrillJoinRelBase;
-import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.apache.calcite.rel.InvalidRelException;
-import org.apache.calcite.rel.core.JoinRelType;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.InvalidRelException;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.core.JoinRelType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.util.Pair;
-
-import com.google.common.collect.Lists;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Join;
+import org.apache.drill.common.logical.data.JoinCondition;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.Project;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.planner.torel.ConversionContext;
 
 /**
  * Logical Join implemented in Drill.
@@ -54,13 +51,14 @@ public class DrillJoinRel extends DrillJoinRelBase implements DrillRel {
   public DrillJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType)  {
     super(cluster, traits, left, right, condition, joinType);
-
+    assert traits.contains(DrillRel.DRILL_LOGICAL);
     RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
   }
 
   public DrillJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType, List<Integer> leftKeys, List<Integer> rightKeys) throws InvalidRelException {
     super(cluster, traits, left, right, condition, joinType);
+    assert traits.contains(DrillRel.DRILL_LOGICAL);
 
     assert (leftKeys != null && rightKeys != null);
     this.leftKeys = leftKeys;

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
index ab70d93..3fa21db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRuleSets.java
@@ -21,9 +21,13 @@ import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
 
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
+import org.apache.calcite.rel.core.RelFactories;
 import org.apache.calcite.rel.rules.AggregateExpandDistinctAggregatesRule;
 import org.apache.calcite.rel.rules.AggregateRemoveRule;
 import org.apache.calcite.rel.rules.FilterAggregateTransposeRule;
+import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
 import org.apache.calcite.rel.rules.JoinPushExpressionsRule;
 import org.apache.calcite.rel.rules.JoinPushThroughJoinRule;
@@ -33,8 +37,6 @@ import org.apache.calcite.rel.rules.ReduceExpressionsRule;
 import org.apache.calcite.rel.rules.SortRemoveRule;
 import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.tools.RuleSet;
-
-import org.apache.calcite.rel.rules.FilterMergeRule;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
 import org.apache.drill.exec.planner.logical.partition.ParquetPruneScanRule;
 import org.apache.drill.exec.planner.logical.partition.PruneScanRule;
@@ -53,13 +55,10 @@ import org.apache.drill.exec.planner.physical.ScreenPrule;
 import org.apache.drill.exec.planner.physical.SortConvertPrule;
 import org.apache.drill.exec.planner.physical.SortPrule;
 import org.apache.drill.exec.planner.physical.StreamAggPrule;
+import org.apache.drill.exec.planner.physical.UnionAllPrule;
 import org.apache.drill.exec.planner.physical.ValuesPrule;
 import org.apache.drill.exec.planner.physical.WindowPrule;
-import org.apache.drill.exec.planner.physical.UnionAllPrule;
 import org.apache.drill.exec.planner.physical.WriterPrule;
-import org.apache.calcite.rel.core.RelFactories;
-import org.apache.calcite.plan.RelOptRule;
-import org.apache.calcite.plan.volcano.AbstractConverter.ExpandConversionRule;
 
 import com.google.common.collect.ImmutableSet;
 import com.google.common.collect.ImmutableSet.Builder;

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
index c9af8af..60f2ebf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.java
@@ -17,10 +17,10 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelTraitDef;
 import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.RelNode;
 
 public class DrillDistributionTraitDef extends RelTraitDef<DrillDistributionTrait>{
   public static final DrillDistributionTraitDef INSTANCE = new DrillDistributionTraitDef();
@@ -72,6 +72,11 @@ public class DrillDistributionTraitDef extends RelTraitDef<DrillDistributionTrai
         return null;
     }
 
+    // It is only possible to apply a distribution trait to a DRILL_PHYSICAL convention.
+    if (rel.getConvention() != Prel.DRILL_PHYSICAL) {
+      return null;
+    }
+
     switch(toDist.getType()){
       // UnionExchange, HashToRandomExchange, OrderedPartitionExchange and BroadcastExchange destroy the ordering property,
       // therefore RelCollation is set to default, which is EMPTY.

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
index a54e350..9072d34 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/explain/NumberingRelWriter.java
@@ -24,16 +24,16 @@ import java.util.Map;
 
 import org.apache.calcite.avatica.util.Spacer;
 import org.apache.calcite.linq4j.Ord;
-import org.apache.calcite.runtime.FlatLists;
-import org.apache.drill.exec.planner.physical.HashJoinPrel;
-import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.planner.physical.explain.PrelSequencer.OpId;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.FlatLists;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.Pair;
+import org.apache.drill.exec.planner.physical.HashJoinPrel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.explain.PrelSequencer.OpId;
 
 import com.google.common.collect.ImmutableList;
 
@@ -78,12 +78,16 @@ class NumberingRelWriter implements RelWriter {
 
     StringBuilder s = new StringBuilder();
     OpId id = ids.get(rel);
-    s.append(String.format("%02d-%02d",id.fragmentId, id.opId));
+    if (id != null) {
+      s.append(String.format("%02d-%02d", id.fragmentId, id.opId));
+    }else{
+      s.append("     ");
+    }
     s.append("  ");
-    if(id.opId == 0){
+
+    if (id != null && id.opId == 0) {
       for(int i =0; i < spacer.get(); i++){ s.append('-');}
     }else{
-
       spacer.spaces(s);
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
index b8acfcf..085f808 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/SchemaUtilites.java
@@ -103,7 +103,7 @@ public class SchemaUtilites {
    */
   public static AbstractSchema unwrapAsDrillSchemaInstance(SchemaPlus schemaPlus)  {
     try {
-      return schemaPlus.unwrap(AbstractSchema.class).getDefaultSchema();
+      return (AbstractSchema) schemaPlus.unwrap(AbstractSchema.class).getDefaultSchema();
     } catch (ClassCastException e) {
       throw UserException.validationError(e)
           .message("Schema [%s] is not a Drill schema.", getSchemaPath(schemaPlus))

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
index 5f90bd6..53b57bc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -25,11 +25,11 @@ import java.util.List;
 import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
-import org.apache.calcite.plan.hep.HepPlanner;
-import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.plan.hep.HepMatchOrder;
+import org.apache.calcite.plan.hep.HepPlanner;
 import org.apache.calcite.plan.hep.HepProgram;
 import org.apache.calcite.plan.hep.HepProgramBuilder;
+import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelShuttleImpl;
 import org.apache.calcite.rel.core.Join;
 import org.apache.calcite.rel.core.Project;
@@ -39,7 +39,6 @@ import org.apache.calcite.rel.core.TableScan;
 import org.apache.calcite.rel.logical.LogicalValues;
 import org.apache.calcite.rel.metadata.CachingRelMetadataProvider;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
-import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
 import org.apache.calcite.rel.rules.JoinToMultiJoinRule;
 import org.apache.calcite.rel.rules.LoptOptimizeJoinRule;
@@ -100,11 +99,11 @@ import org.apache.drill.exec.util.Pointer;
 import org.apache.drill.exec.work.foreman.ForemanSetupException;
 import org.apache.drill.exec.work.foreman.SqlUnsupportedException;
 import org.apache.drill.exec.work.foreman.UnsupportedRelOperatorException;
+import org.slf4j.Logger;
 
 import com.fasterxml.jackson.core.JsonProcessingException;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.slf4j.Logger;
 
 public class DefaultSqlHandler extends AbstractSqlHandler {
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DefaultSqlHandler.class);
@@ -260,7 +259,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
     RelTraitSet traits = drel.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
     Prel phyRelNode;
     try {
-      phyRelNode = (Prel) planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
+      final RelNode relNode = planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
+      phyRelNode = (Prel) relNode.accept(PrelFinalizable.SHUTTLE);
     } catch (RelOptPlanner.CannotPlanException ex) {
       logger.error(ex.getMessage());
 
@@ -282,7 +282,8 @@ public class DefaultSqlHandler extends AbstractSqlHandler {
       queryOptions.setOption(OptionValue.createBoolean(OptionValue.OptionType.QUERY, PlannerSettings.HASHAGG.getOptionName(), false));
 
       try {
-        phyRelNode = (Prel) planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
+        final RelNode relNode = planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
+        phyRelNode = (Prel) relNode.accept(PrelFinalizable.SHUTTLE);
       } catch (RelOptPlanner.CannotPlanException ex) {
         logger.error(ex.getMessage());
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/PrelFinalizable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/PrelFinalizable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/PrelFinalizable.java
new file mode 100644
index 0000000..1eadb09
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/PrelFinalizable.java
@@ -0,0 +1,47 @@
+/*
+ * 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.drill.exec.planner.sql.handlers;
+
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttle;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.drill.exec.planner.physical.Prel;
+
+/**
+ * A marker interface that means that this node should be finalized before execution planning. This allows calculations
+ * not relevant to planning to be done only once.
+ */
+public interface PrelFinalizable {
+  Prel finalizeRel();
+
+  /**
+   * A shuttle designed to finalize all RelNodes.
+   */
+  RelShuttle SHUTTLE = new RelShuttleImpl() {
+
+    @Override
+    public RelNode visit(RelNode other) {
+      if (other instanceof PrelFinalizable) {
+        return ((PrelFinalizable) other).finalizeRel();
+      } else {
+        return super.visit(other);
+      }
+    }
+
+  };
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
index 0db51ab1..2403252 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractSchema.java
@@ -29,9 +29,7 @@ import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.schema.Table;
-
 import org.apache.drill.common.exceptions.UserException;
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.dotdrill.View;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 
@@ -88,7 +86,7 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
    *
    * @return Return the default schema where tables are created or retrieved from.
    */
-  public AbstractSchema getDefaultSchema() {
+  public Schema getDefaultSchema() {
     return this;
   }
 
@@ -152,7 +150,7 @@ public abstract class AbstractSchema implements Schema, SchemaPartitionExplorer,
   }
 
   @Override
-  public AbstractSchema getSubSchema(String name) {
+  public Schema getSubSchema(String name) {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
index 822a210..33870aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/AbstractStoragePlugin.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.calcite.plan.RelOptRule;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
@@ -45,7 +46,7 @@ public abstract class AbstractStoragePlugin implements StoragePlugin{
   }
 
   @Override
-  public Set<StoragePluginOptimizerRule> getOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
+  public Set<? extends RelOptRule> getOptimizerRules(OptimizerRulesContext optimizerRulesContext) {
     return ImmutableSet.of();
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
index eef63a2..375116d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePlugin.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 import java.util.List;
 import java.util.Set;
 
+import org.apache.calcite.plan.RelOptRule;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.StoragePluginConfig;
@@ -32,7 +33,7 @@ public interface StoragePlugin extends SchemaFactory {
 
   public boolean supportsWrite();
 
-  public Set<StoragePluginOptimizerRule> getOptimizerRules(OptimizerRulesContext optimizerContext);
+  public Set<? extends RelOptRule> getOptimizerRules(OptimizerRulesContext optimizerContext);
 
   /**
    * Get the physical scan operator for the particular GroupScan (read) node.

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
index d58f9a8..6eb65e6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/StoragePluginRegistry.java
@@ -31,9 +31,9 @@ import java.util.Set;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.tools.RuleSet;
-
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -43,11 +43,8 @@ import org.apache.drill.common.util.PathScanner;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.DrillbitStartupException;
 import org.apache.drill.exec.ops.OptimizerRulesContext;
-import org.apache.drill.exec.ops.QueryContext;
-import org.apache.drill.exec.ops.ViewExpansionContext;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
 import org.apache.drill.exec.planner.logical.StoragePlugins;
-import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
@@ -57,7 +54,6 @@ import org.apache.drill.exec.store.sys.PStore;
 import org.apache.drill.exec.store.sys.PStoreConfig;
 import org.apache.drill.exec.store.sys.SystemTablePlugin;
 import org.apache.drill.exec.store.sys.SystemTablePluginConfig;
-import org.apache.calcite.plan.RelOptRule;
 
 import com.google.common.base.Charsets;
 import com.google.common.base.Joiner;
@@ -296,7 +292,7 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     // query registered engines for optimizer rules and build the storage plugin RuleSet
     Builder<RelOptRule> setBuilder = ImmutableSet.builder();
     for (StoragePlugin plugin : this.plugins.values()) {
-      Set<StoragePluginOptimizerRule> rules = plugin.getOptimizerRules(optimizerRulesContext);
+      Set<? extends RelOptRule> rules = plugin.getOptimizerRules(optimizerRulesContext);
       if (rules != null && rules.size() > 0) {
         setBuilder.addAll(rules);
       }

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
index 722638a..e502e99 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SubSchemaWrapper.java
@@ -18,14 +18,13 @@
 package org.apache.drill.exec.store;
 
 import java.util.Collection;
+import java.util.Collections;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.calcite.schema.Function;
 import org.apache.calcite.schema.Schema;
 import org.apache.calcite.schema.Table;
-
-import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.planner.logical.CreateTableEntry;
 
 import com.google.common.collect.ImmutableList;
@@ -49,11 +48,17 @@ public class SubSchemaWrapper extends AbstractSchema {
                                            List<String> partitionColumns,
                                            List<String> partitionValues
   ) throws PartitionNotFoundException {
-    return getDefaultSchema().getSubPartitions(table, partitionColumns, partitionValues);
+    Schema defaultSchema = getDefaultSchema();
+    if (defaultSchema instanceof AbstractSchema) {
+      return ((AbstractSchema) defaultSchema).getSubPartitions(table, partitionColumns, partitionValues);
+    } else {
+      return Collections.EMPTY_LIST;
+    }
+
   }
 
   @Override
-  public AbstractSchema getDefaultSchema() {
+  public Schema getDefaultSchema() {
     return innerSchema.getDefaultSchema();
   }
 
@@ -73,7 +78,7 @@ public class SubSchemaWrapper extends AbstractSchema {
   }
 
   @Override
-  public AbstractSchema getSubSchema(String name) {
+  public Schema getSubSchema(String name) {
     return innerSchema.getSubSchema(name);
   }
 


[11/15] drill git commit: Reduce test time by not waiting 2 seconds for newly submitted tasks before shutting down RPC thread pool.

Posted by ja...@apache.org.
Reduce test time by not waiting 2 seconds for newly submitted tasks before shutting down RPC thread pool.


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

Branch: refs/heads/master
Commit: 71086bf82b5dd3aca3674147df3fd97455883fae
Parents: fe07b6c
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Sep 13 18:51:46 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:33 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/rpc/BasicServer.java  |  3 +-
 .../org/apache/drill/exec/work/WorkManager.java | 32 ++++++++++++++------
 2 files changed, 25 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/71086bf8/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
index c20afc0..00bbcb4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/BasicServer.java
@@ -31,6 +31,7 @@ import io.netty.handler.timeout.ReadTimeoutHandler;
 import java.io.IOException;
 import java.net.BindException;
 import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.drill.common.exceptions.UserException;
 import org.apache.drill.exec.exception.DrillbitStartupException;
@@ -215,7 +216,7 @@ public abstract class BasicServer<T extends EnumLite, C extends RemoteConnection
   @Override
   public void close() throws IOException {
     try {
-      eventLoopGroup.shutdownGracefully().get();
+      eventLoopGroup.shutdownGracefully(0, 2, TimeUnit.SECONDS).get();
     } catch (final InterruptedException | ExecutionException e) {
       logger.warn("Failure while shutting down {}. ", this.getClass().getName(), e);
 

http://git-wip-us.apache.org/repos/asf/drill/blob/71086bf8/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
index 5939113..8209277 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/WorkManager.java
@@ -168,16 +168,26 @@ public class WorkManager implements AutoCloseable {
 
   @Override
   public void close() throws Exception {
-    try {
-      if (executor != null) {
-        executor.awaitTermination(1, TimeUnit.SECONDS);
-      }
-    } catch (final InterruptedException e) {
-      logger.warn("Executor interrupted while awaiting termination");
 
-      // Preserve evidence that the interruption occurred so that code higher up on the call stack can learn of the
-      // interruption and respond to it if it wants to.
-      Thread.currentThread().interrupt();
+    if (executor != null) {
+      executor.shutdown(); // Disable new tasks from being submitted
+      try {
+        // Wait a while for existing tasks to terminate
+        if (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+          executor.shutdownNow(); // Cancel currently executing tasks
+          // Wait a while for tasks to respond to being cancelled
+          if (!executor.awaitTermination(1, TimeUnit.SECONDS)) {
+            logger.error("Pool did not terminate");
+          }
+        }
+      } catch (InterruptedException ie) {
+        logger.warn("Executor interrupted while awaiting termination");
+
+        // (Re-)Cancel if current thread also interrupted
+        executor.shutdownNow();
+        // Preserve interrupt status
+        Thread.currentThread().interrupt();
+      }
     }
 
     if (!runningFragments.isEmpty()) {
@@ -191,6 +201,10 @@ public class WorkManager implements AutoCloseable {
     }
   }
 
+  private void shutdownAndAwaitTermination(long timeInSeconds) {
+
+  }
+
   public DrillbitContext getContext() {
     return dContext;
   }


[13/15] drill git commit: DRILL-3589: Update JDBC driver to shade and minimize dependencies.

Posted by ja...@apache.org.
DRILL-3589: Update JDBC driver to shade and minimize dependencies.

Update build process to use shading, stop using Proguard.
Add simple integration test that verifies that the JDBC driver works
correctly with a clean classpath.


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

Branch: refs/heads/master
Commit: 4e3b7dc0333a01e72d0ea9256331ea1e1dd51181
Parents: e5f529b
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Aug 16 11:46:26 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:34 2015 -0700

----------------------------------------------------------------------
 .../apache/drill/common/util/PathScanner.java   |  49 ++-
 distribution/pom.xml                            |   5 +
 .../drill/exec/compile/QueryClassLoader.java    |   2 +-
 .../drill/exec/server/BootStrapContext.java     |  10 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   2 +-
 exec/jdbc-all/pom.xml                           | 411 +++++++++++--------
 .../apache/drill/jdbc/DrillbitClassLoader.java  | 106 +++++
 .../org/apache/drill/jdbc/ITTestShadedJar.java  | 189 +++++++++
 exec/jdbc-all/src/test/resources/logback.xml    |  54 +++
 .../main/java/org/apache/drill/jdbc/Driver.java |  14 +-
 10 files changed, 620 insertions(+), 222 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/common/src/main/java/org/apache/drill/common/util/PathScanner.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/PathScanner.java b/common/src/main/java/org/apache/drill/common/util/PathScanner.java
index 16537b6..a44bdf2 100644
--- a/common/src/main/java/org/apache/drill/common/util/PathScanner.java
+++ b/common/src/main/java/org/apache/drill/common/util/PathScanner.java
@@ -35,7 +35,6 @@ import org.reflections.Reflections;
 import org.reflections.scanners.ResourcesScanner;
 import org.reflections.scanners.SubTypesScanner;
 import org.reflections.scanners.TypeAnnotationsScanner;
-import org.reflections.util.ClasspathHelper;
 import org.reflections.util.ConfigurationBuilder;
 
 import com.google.common.base.Stopwatch;
@@ -143,33 +142,31 @@ public class PathScanner {
                  resourcePathname);
     final Set<URL> resultUrlSet = Sets.newHashSet();
 
-    final ClassLoader[] netLoaders = ClasspathHelper.classLoaders(classLoaders);
-    for (ClassLoader classLoader : netLoaders) {
-      try {
-        final Enumeration<URL> resourceUrls =
-            classLoader.getResources(resourcePathname);
-        while (resourceUrls.hasMoreElements()) {
-          final URL resourceUrl = resourceUrls.nextElement();
-          logger.trace( "- found a(n) {} at {}.", resourcePathname, resourceUrl );
-
-          int index = resourceUrl.toExternalForm().lastIndexOf(resourcePathname);
-          if (index != -1 && returnRootPathname) {
-            final URL classpathRootUrl =
-                new URL(resourceUrl.toExternalForm().substring(0, index));
-            resultUrlSet.add(classpathRootUrl);
-            logger.debug( "- collected resource's classpath root URL {}.",
-                          classpathRootUrl );
-          } else {
-            resultUrlSet.add(resourceUrl);
-            logger.debug( "- collected resource URL {}.", resourceUrl );
-          }
-        }
-      } catch (IOException e) {
-        if (Reflections.log != null) {
-          Reflections.log.error(
-              "Error scanning for resources named " + resourcePathname, e);
+    final ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+    try {
+      final Enumeration<URL> resourceUrls =
+          classLoader.getResources(resourcePathname);
+      while (resourceUrls.hasMoreElements()) {
+        final URL resourceUrl = resourceUrls.nextElement();
+        logger.trace("- found a(n) {} at {}.", resourcePathname, resourceUrl);
+
+        int index = resourceUrl.toExternalForm().lastIndexOf(resourcePathname);
+        if (index != -1 && returnRootPathname) {
+          final URL classpathRootUrl =
+              new URL(resourceUrl.toExternalForm().substring(0, index));
+          resultUrlSet.add(classpathRootUrl);
+          logger.debug("- collected resource's classpath root URL {}.",
+              classpathRootUrl);
+        } else {
+          resultUrlSet.add(resourceUrl);
+          logger.debug("- collected resource URL {}.", resourceUrl);
         }
       }
+    } catch (IOException e) {
+      if (Reflections.log != null) {
+        Reflections.log.error(
+            "Error scanning for resources named " + resourcePathname, e);
+      }
     }
 
     return resultUrlSet;

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/distribution/pom.xml
----------------------------------------------------------------------
diff --git a/distribution/pom.xml b/distribution/pom.xml
index c08efc7..f4a17ac 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -158,6 +158,11 @@
           </exclusions>
         </dependency>
         <dependency>
+          <groupId>org.apache.drill.contrib</groupId>
+          <artifactId>drill-jdbc-storage</artifactId>
+          <version>${project.version}</version>
+        </dependency>
+        <dependency>
           <groupId>org.apache.drill.contrib.storage-hive</groupId>
           <artifactId>drill-storage-hive-core</artifactId>
           <version>${project.version}</version>

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
index c4eaae8..3df8f84 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/QueryClassLoader.java
@@ -74,7 +74,7 @@ public class QueryClassLoader extends URLClassLoader {
   private ConcurrentMap<String, byte[]> customClasses = new MapMaker().concurrencyLevel(4).makeMap();
 
   public QueryClassLoader(DrillConfig config, OptionManager sessionOptions) {
-    super(new URL[0]);
+    super(new URL[0], Thread.currentThread().getContextClassLoader());
     compilerSelector = new ClassCompilerSelector(config, sessionOptions);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
index 120c0d0..4530dba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/BootStrapContext.java
@@ -30,9 +30,9 @@ import org.apache.drill.exec.rpc.TransportCheck;
 
 import com.codahale.metrics.MetricRegistry;
 
-// TODO:  Doc.  What kind of context?  (For what aspects, RPC?  What kind of data?)
+
 public class BootStrapContext implements Closeable {
-  //private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class);
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BootStrapContext.class);
 
   private final DrillConfig config;
   private final EventLoopGroup loop;
@@ -70,7 +70,11 @@ public class BootStrapContext implements Closeable {
 
   @Override
   public void close() {
-    DrillMetrics.resetMetrics();
+    try {
+      DrillMetrics.resetMetrics();
+    } catch (Error | Exception e) {
+      logger.warn("failure resetting metrics.", e);
+    }
     loop.shutdownGracefully();
     allocator.close();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
index 1381949..cb137ee 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/BaseTestQuery.java
@@ -300,7 +300,7 @@ public class BaseTestQuery extends ExecTest {
     QueryTestUtil.testWithListener(client, type, query, resultListener);
   }
 
-  protected static void testNoResult(String query, Object... args) throws Exception {
+  public static void testNoResult(String query, Object... args) throws Exception {
     testNoResult(1, query, args);
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index ed49cfb..5ef6e07 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -28,99 +28,29 @@
   <name>JDBC JAR with all dependencies</name>
 
   <dependencies>
-  
-    <!-- start parent dependencies -->
-    <dependency>
-      <groupId>io.netty</groupId>
-      <artifactId>netty-handler</artifactId>
-      <version>4.0.27.Final</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
-      <version>14.0.1</version>
-      <scope>provided</scope>
-    </dependency>
 
     <dependency>
       <groupId>org.slf4j</groupId>
       <artifactId>slf4j-api</artifactId>
       <version>${dep.slf4j.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>jul-to-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>jcl-over-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>log4j-over-slf4j</artifactId>
-      <version>${dep.slf4j.version}</version>
-      <scope>provided</scope>
     </dependency>
 
     <dependency>
-      <groupId>com.googlecode.jmockit</groupId>
-      <artifactId>jmockit</artifactId>
-      <version>1.3</version>
-      <scope>provided</scope>
-    </dependency>
-    
-    <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <version>${dep.junit.version}</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>org.mockito</groupId>
-      <artifactId>mockito-core</artifactId>
-      <version>1.9.5</version>
-      <scope>provided</scope>
-    </dependency>
-
-    <dependency>
-      <groupId>ch.qos.logback</groupId>
-      <artifactId>logback-classic</artifactId>
-      <version>1.0.13</version>
-      <scope>provided</scope>
-    </dependency>
-    
-    <dependency>
-      <groupId>de.huxhorn.lilith</groupId>
-      <artifactId>de.huxhorn.lilith.logback.appender.multiplex-classic</artifactId>
-      <version>0.9.44</version>
-      <scope>provided</scope>
-    </dependency>
-        
-    <!-- end parent dependencies -->
-            
-    <dependency>
-      <groupId>net.hydromatic</groupId>
-      <artifactId>optiq-avatica</artifactId>
-      <version>0.9-drill-r20</version>
-      <scope>provided</scope>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <version>${project.version}</version>
+      <exclusions>
+        <exclusion>
+          <artifactId>javassist</artifactId>
+          <groupId>javassist</groupId>
+        </exclusion>
+      </exclusions>
     </dependency>
 
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-java-exec</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <groupId>log4j</groupId>
@@ -208,7 +138,6 @@
       <groupId>org.apache.drill</groupId>
       <artifactId>drill-common</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <artifactId>javassist</artifactId>
@@ -220,7 +149,6 @@
       <groupId>org.apache.drill.exec</groupId>
       <artifactId>drill-jdbc</artifactId>
       <version>${project.version}</version>
-      <scope>provided</scope>
       <exclusions>
         <exclusion>
           <artifactId>drill-storage-hive-core</artifactId>
@@ -237,148 +165,267 @@
       <groupId>org.codehaus.janino</groupId>
       <artifactId>janino</artifactId>
       <version>2.6.1</version>
-      <scope>provided</scope>
+      <scope>test</scope>
     </dependency>
     <!-- Specify xalan and xerces versions to avoid setXIncludeAware error. -->
     <dependency>
       <groupId>xerces</groupId>
       <artifactId>xercesImpl</artifactId>
-      <scope>provided</scope>
       <exclusions>
-        <exclusion>
-          <groupId>xml-apis</groupId>
-          <artifactId>xml-apis</artifactId>
-        </exclusion>
+        <!-- <exclusion> -->
+        <!-- <groupId>xml-apis</groupId> -->
+        <!-- <artifactId>xml-apis</artifactId> -->
+        <!-- </exclusion> -->
       </exclusions>
     </dependency>
     <dependency>
       <groupId>xalan</groupId>
       <artifactId>xalan</artifactId>
-      <scope>provided</scope>
       <exclusions>
-        <exclusion>
-          <groupId>xml-apis</groupId>
-          <artifactId>xml-apis</artifactId>
-        </exclusion>
+        <!-- <exclusion> -->
+        <!-- <groupId>xml-apis</groupId> -->
+        <!-- <artifactId>xml-apis</artifactId> -->
+        <!-- </exclusion> -->
       </exclusions>
     </dependency>
+
+
+    <!-- Test Dependencies -->
     <dependency>
-      <groupId>ch.qos.logback</groupId>
-      <artifactId>logback-classic</artifactId>
-      <version>1.0.13</version>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
     <dependency>
-      <groupId>ch.qos.logback</groupId>
-      <artifactId>logback-core</artifactId>
-      <version>1.0.13</version>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
       <scope>provided</scope>
     </dependency>
+
   </dependencies>
 
   <build>
     <plugins>
+     
+      <plugin>
+        <artifactId>maven-antrun-plugin</artifactId>
+        <executions>
+          <execution>
+            <id>define-classpath</id>
+            <phase>integration-test</phase>
+            <goals>
+              <goal>run</goal>
+            </goals>
+            <configuration>
+              <exportAntProperties>true</exportAntProperties>
+              <target>
+                <property name="app.class.path" refid="maven.test.classpath" />
+              </target>
+            </configuration>
+          </execution>
+        </executions>
+      </plugin>
 
       <plugin>
-        <groupId>com.github.wvengen</groupId>
-        <artifactId>proguard-maven-plugin</artifactId>
-        <version>2.0.7</version>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-failsafe-plugin</artifactId>
+        <version>2.18.1</version>
         <executions>
           <execution>
-            <phase>package</phase>
             <goals>
-              <goal>proguard</goal>
+              <goal>integration-test</goal>
+              <goal>verify</goal>
             </goals>
           </execution>
         </executions>
-        <dependencies>
-          <dependency>
-            <groupId>net.sf.proguard</groupId>
-            <artifactId>proguard-base</artifactId>
-            <version>5.0</version>
-            <scope>runtime</scope>
-          </dependency>
-        </dependencies>
+
         <configuration>
-          <proguardVersion>5.0</proguardVersion>
-          <obfuscate>false</obfuscate>
-          <includeDependencyInjar>true</includeDependencyInjar>
-          <outjar>${project.build.finalName}-dirty.jar</outjar>
-          <outputDirectory>${project.build.directory}</outputDirectory>
-          <maxMemory>6g</maxMemory>
-          <options>
-            <option>-dontobfuscate</option>
-            <option>-dontoptimize</option>
-            <option>-ignorewarnings</option>
-            <option>-keep class org.apache.drill.exec.proto.** { *; }</option>
-            <option>-keep class org.apache.drill.common.types.** { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.Driver { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillConnection { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillStatement { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.DrillResultSet { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.impl.DrillJdbc40Factory { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.impl.DrillJdbc41Factory { *; }</option>
-            <option>-keep class org.apache.drill.jdbc.proxy.TracingProxyDriver { *; }</option>
-            <option>-keep class org.apache.drill.common.config.CommonConstants { *; }</option>
-            <option>-keep class org.apache.drill.common.config.ConfigProvider { *; }</option>
-            <option>-keep class org.apache.drill.common.config.DrillConfig { *; }</option>
-            <option>-keep class org.apache.drill.common.config.NestedConfig { *; }</option>
+          <argLine>-Xms512m -Xmx3g -Ddrill.exec.http.enabled=false -Djava.net.preferIPv4Stack=true
+            -Ddrill.exec.sys.store.provider.local.write=false
+            -Dorg.apache.drill.exec.server.Drillbit.system_options="org.apache.drill.exec.compile.ClassTransformer.scalar_replacement=on"
+            -XX:MaxPermSize=256M -XX:MaxDirectMemorySize=3072M
+            -XX:+CMSClassUnloadingEnabled -ea</argLine>
+          <additionalClasspathElements>
+            <additionalClasspathElements>${settings.localRepository}/junit/junit/4.11/junit-4.11.jar</additionalClasspathElements>
+            <additionalClasspathElements>${settings.localRepository}/org/hamcrest/hamcrest-core/1.3/hamcrest-core-1.3.jar</additionalClasspathElements>
+            <additionalClasspathElements>${project.build.directory}/test-classes/</additionalClasspathElements>
+          </additionalClasspathElements>
+          <classpathDependencyExcludes>
+            <classpathDependencyExcludes>*:*</classpathDependencyExcludes>
+          </classpathDependencyExcludes>
 
-            <option>-keep class ch.qos.logback.** { *; }</option>
-            <option>-keep class org.slf4j.** { *; }</option>
-            <option>-keep class * implements com.fasterxml.jackson.databind.cfg.ConfigFeature { *; }</option>
-            <option>-keep class * implements com.fasterxml.jackson.databind.jsontype.TypeIdResolver { *; }</option>
-            <!-- do not mess with enums, Java doesn't like it -->
-            <option>-keep enum ** { *; }</option>
-          </options>
-          <exclusions>
-            <exclusion>
-              <groupId>org.slf4j</groupId>
-              <artifactId>jcl-over-slf4j</artifactId>
-            </exclusion>
-            <exclusion>
-              <groupId>net.hydromatic</groupId>
-              <artifactId>eigenbase-properties</artifactId>
-            </exclusion>
-          </exclusions>
-          <libs>
-            <lib>${java.home}/lib/rt.jar</lib>
-            <lib>${java.home}/lib/jsse.jar</lib>
-          </libs>
+          <systemPropertyVariables>
+            <java.io.tmpdir>${project.build.directory}</java.io.tmpdir>
+            <app.class.path>${app.class.path}</app.class.path>
+            <project.version>${project.version}</project.version>
+          </systemPropertyVariables>
         </configuration>
       </plugin>
 
       <plugin>
-        <artifactId>maven-antrun-plugin</artifactId>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <version>2.4.1</version>
         <executions>
           <execution>
             <phase>package</phase>
-            <configuration>
-              <target>
-                <delete dir="${project.build.directory}/dirty"/>
-                <mkdir dir="${project.build.directory}/dirty"/>
-                <unzip src="${project.build.directory}/${project.build.finalName}-dirty.jar" dest="${project.build.directory}/dirty">
-                  <patternset>
-                    <exclude name="**/*.java"/>
-                    <exclude name="org.codehaus.commons.compiler.properties"/> <!-- This leads jad-ui to not parse classes under 'org/' -->
-                    <exclude name="**/*.SF"/>
-                    <exclude name="**/*.RSA"/>
-                    <exclude name="**/*.DSA"/>
-                    <exclude name="META-INF/services/*"/>
-                    <exclude name="META-INF/*.SF"/>
-                    <exclude name="META-INF/*.RSA"/>
-                    <exclude name="META-INF/*.DSA"/>
-                  </patternset>
-                </unzip>
-                <jar destfile="${project.build.directory}/${project.build.finalName}.jar" basedir="${project.build.directory}/dirty"/>
-                <delete dir="${project.build.directory}/dirty"/>
-              </target>
-            </configuration>
             <goals>
-              <goal>run</goal>
+              <goal>shade</goal>
             </goals>
           </execution>
         </executions>
+        <configuration>
+          <shadedArtifactAttached>false</shadedArtifactAttached>
+          <createDependencyReducedPom>true</createDependencyReducedPom>
+          <dependencyReducedPomLocation>${project.build.directory}/dependency-reduced-pom.xml</dependencyReducedPomLocation>
+          <minimizeJar>false</minimizeJar>
+          
+          <artifactSet>
+            <includes>
+              <include>*:*</include>
+            </includes>
+            <excludes>
+              <exclude>org.slf4j:jcl-over-slf4j</exclude>
+              <exclude>com.dyuproject.protostuff:*</exclude>
+              <exclude>org.apache.calcite:*</exclude>
+              <exclude>org.pentaho:*</exclude>
+              <exclude>org.msgpack:*</exclude>
+              <exclude>com.googlecode.json-simple:*</exclude>
+              <exclude>dom4j:*</exclude>
+              <exclude>org.hibernate:*</exclude>
+              <exclude>javax.validation:*</exclude>
+              <exclude>antlr:*</exclude>
+              <exclude>org.ow2.asm:*</exclude>
+              <exclude>com.univocity:*</exclude>
+              <exclude>net.sf.jpam:*</exclude>
+              <exclude>com.twitter:*</exclude>
+              <exclude>javax.inject:*</exclude>
+              <exclude>com.beust:*</exclude>
+              <exclude>org.codehaus.jackson:*</exclude>
+              <exclude>jline:*</exclude>
+              <exclude>io.netty:netty:jar:3.7.0.Final</exclude>
+              <exclude>org.xerial.snappy:*</exclude>
+              <exclude>org.apache.avro:*</exclude>
+              <exclude>org.tukaani:*</exclude>
+              <exclude>org.apache.velocity:*</exclude>
+              <exclude>net.hydromatic:linq4j</exclude>
+              <exclude>org.codehaus.janino:*</exclude>
+              <exclude>org.mortbay.jetty:*</exclude>
+              <exclude>org.slf4j:jul-to-slf4j</exclude>
+              <exclude>org.slf4j:log4j-over-slf4j</exclude>
+              <exclude>org.hamcrest:hamcrest-core</exclude>
+              <exclude>org.mockito:mockito-core</exclude>
+              <exclude>org.objenesis:objenesis</exclude>
+            </excludes>
+          </artifactSet>
+          <relocations>
+            <!-- Relocate Drill classes to minimize classloader hell. -->
+            <relocation><pattern>org.apache.drill.exec.</pattern><shadedPattern>oadd.org.apache.drill.exec.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.drill.common.</pattern><shadedPattern>oadd.org.apache.drill.common.</shadedPattern></relocation>
+            
+            <!-- Move dependencies out of path -->
+            <relocation><pattern>antlr.</pattern><shadedPattern>oadd.antlr.</shadedPattern></relocation>
+            <relocation><pattern>antlr.</pattern><shadedPattern>oadd.antlr.</shadedPattern></relocation>
+            <relocation><pattern>io.</pattern><shadedPattern>oadd.io.</shadedPattern></relocation>
+            <relocation><pattern>javacc.</pattern><shadedPattern>oadd.javacc.</shadedPattern></relocation>
+            <relocation><pattern>java_cup.</pattern><shadedPattern>oadd.java_cup.</shadedPattern></relocation>
+            <relocation><pattern>javassist.</pattern><shadedPattern>oadd.javassist.</shadedPattern></relocation>
+            <relocation><pattern>jline.</pattern><shadedPattern>oadd.jline.</shadedPattern></relocation>
+            <relocation><pattern>license.</pattern><shadedPattern>oadd.license.</shadedPattern></relocation>
+            <relocation><pattern>net.</pattern><shadedPattern>oadd.net.</shadedPattern></relocation>
+            <relocation><pattern>parquet.</pattern><shadedPattern>oadd.parquet.</shadedPattern></relocation>
+            <relocation><pattern>test.</pattern><shadedPattern>oadd.test.</shadedPattern></relocation>
+            <relocation><pattern>trax.</pattern><shadedPattern>oadd.trax.</shadedPattern></relocation>
+            <relocation><pattern>org.antlr.</pattern><shadedPattern>oadd.org.antlr.</shadedPattern></relocation>
+            <relocation><pattern>org.codehaus.</pattern><shadedPattern>oadd.org.codehaus.</shadedPattern></relocation>
+            <relocation><pattern>org.eigenbase.</pattern><shadedPattern>oadd.org.eigenbase.</shadedPattern></relocation>
+            <relocation><pattern>org.hamcrest.</pattern><shadedPattern>oadd.org.hamcrest.</shadedPattern></relocation>
+            <relocation><pattern>org.jboss.</pattern><shadedPattern>oadd.org.jboss.</shadedPattern></relocation>
+            <relocation><pattern>org.joda.</pattern><shadedPattern>oadd.org.joda.</shadedPattern></relocation>
+            <relocation><pattern>org.json.</pattern><shadedPattern>oadd.org.json.</shadedPattern></relocation>
+            <relocation><pattern>org.mockito.</pattern><shadedPattern>oadd.org.mockito.</shadedPattern></relocation>
+            <relocation><pattern>org.msgpack.</pattern><shadedPattern>oadd.org.msgpack.</shadedPattern></relocation>
+            <relocation><pattern>org.objectweb.</pattern><shadedPattern>oadd.org.objectweb.</shadedPattern></relocation>
+            <relocation><pattern>org.objensis.</pattern><shadedPattern>oadd.org.objensis.</shadedPattern></relocation>
+            <relocation><pattern>org.pentaho.</pattern><shadedPattern>oadd.org.pentaho.</shadedPattern></relocation>
+            <relocation><pattern>org.reflections.</pattern><shadedPattern>oadd.org.reflections.</shadedPattern></relocation>
+            <relocation><pattern>org.tukaani.</pattern><shadedPattern>oadd.org.tukaani.</shadedPattern></relocation>
+            <relocation><pattern>org.xerial.</pattern><shadedPattern>oadd.org.xerial.</shadedPattern></relocation>
+            <relocation><pattern>com.beust.</pattern><shadedPattern>oadd.com.beust.</shadedPattern></relocation>
+            <relocation><pattern>com.carrotsearch.</pattern><shadedPattern>oadd.com.carrotsearch.</shadedPattern></relocation>
+            <relocation><pattern>com.codahale.</pattern><shadedPattern>oadd.com.codahale.</shadedPattern></relocation>
+            <relocation><pattern>com.dyuproject.</pattern><shadedPattern>oadd.com.dyuproject.</shadedPattern></relocation>
+            <relocation><pattern>com.fasterxml.</pattern><shadedPattern>oadd.com.fasterxml.</shadedPattern></relocation>
+            <relocation><pattern>com.google.</pattern><shadedPattern>oadd.com.google.</shadedPattern></relocation>
+            <relocation><pattern>com.thoughtworks.</pattern><shadedPattern>oadd.com.thoughtworks.</shadedPattern></relocation>
+            <relocation><pattern>com.typesafe.</pattern><shadedPattern>oadd.com.typesafe.</shadedPattern></relocation>
+            <relocation><pattern>com.univocity.</pattern><shadedPattern>oadd.com.univocity.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.avro.</pattern><shadedPattern>oadd.org.apache.avro.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.bcel.</pattern><shadedPattern>oadd.org.apache.bcel.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.calcite.</pattern><shadedPattern>oadd.org.apache.calcite.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.commons.</pattern><shadedPattern>oadd.org.apache.commons.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.curator.</pattern><shadedPattern>oadd.org.apache.curator.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.html.</pattern><shadedPattern>oadd.org.apache.html.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.jute.</pattern><shadedPattern>oadd.org.apache.jute.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.log4j.</pattern><shadedPattern>oadd.org.apache.log4j.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.regexp.</pattern><shadedPattern>oadd.org.apache.regexp.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.velocity.</pattern><shadedPattern>oadd.org.apache.velocity.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.wml.</pattern><shadedPattern>oadd.org.apache.wml.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.xalan.</pattern><shadedPattern>oadd.org.apache.xalan.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.xerces.</pattern><shadedPattern>oadd.org.apache.xerces.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.xml.</pattern><shadedPattern>oadd.org.apache.xml.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.xmlcommons.</pattern><shadedPattern>oadd.org.apache.xmlcommons.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.xpath.</pattern><shadedPattern>oadd.org.apache.xpath.</shadedPattern></relocation>
+            <relocation><pattern>org.apache.zookeeper.</pattern><shadedPattern>oadd.org.apache.zookeeper.</shadedPattern></relocation>
+          </relocations>
+          <transformers>
+            <transformer
+              implementation="org.apache.maven.plugins.shade.resource.AppendingTransformer">
+              <resource>drill-module.conf</resource>
+            </transformer>
+          </transformers>
+         <filters>
+           <filter>
+             <artifact>*:*</artifact>
+             <excludes>
+                <exclude>**/logback.xml</exclude>
+                <exclude>**/LICENSE.txt</exclude>
+                <exclude>**/*.java</exclude>
+                <exclude>**/META-INF/**</exclude>
+                <exclude>**/org.codehaus.commons.compiler.properties</exclude>
+                <exclude>**/*.SF</exclude>
+                <exclude>**/*.RSA</exclude>
+                <exclude>**/*.DSA</exclude>
+                <exclude>javax/**</exclude>
+                <exclude>rest/**</exclude>
+                <exclude>*.tokens</exclude>
+                <exclude>com/google/common/math</exclude>
+                <exclude>com/google/common/net</exclude>
+                <exclude>com/google/common/primitives</exclude>
+                <exclude>com/google/common/reflect</exclude>
+                <exclude>com/google/common/util</exclude>
+                <exclude>com/google/common/cache</exclude>
+                <exclude>com/google/common/collect/Tree*</exclude>
+                <exclude>com/google/common/collect/Standard*</exclude>
+                <exclude>org/apache/drill/exec/expr/annotations/**</exclude>
+                <exclude>org/apache/drill/exec/expr/fn/**</exclude>
+                <exclude>org/apache/drill/exec/proto/beans/**</exclude>
+                <exclude>org/apache/drill/exec/compile/**</exclude>
+                <exclude>org/apache/drill/exec/planner/**</exclude>
+                <exclude>org/apache/drill/exec/physical/**</exclude>
+                <exclude>org/apache/drill/exec/store/**</exclude>
+                <exclude>org/apache/drill/exec/server/rest/**</exclude>
+                <exclude>org/apache/drill/exec/rpc/data/**</exclude>
+                <exclude>org/apache/drill/exec/rpc/control/**</exclude>
+                <exclude>org/apache/drill/exec/work/**</exclude>
+              </excludes>
+           </filter>
+         </filters>
+        </configuration>
       </plugin>
 
     </plugins>
@@ -400,11 +447,13 @@
   </pluginRepositories>
 
   <profiles>
-    <!-- mondrian data includes 10s of MBs of JSON file
-        if you want to include them run maven with -Pwith-mondrian-data -->
+    <!-- mondrian data includes 10s of MBs of JSON file if you want to include 
+      them run maven with -Pwith-mondrian-data -->
     <profile>
       <id>with-mondrian-data</id>
-      <activation><activeByDefault>false</activeByDefault></activation>
+      <activation>
+        <activeByDefault>false</activeByDefault>
+      </activation>
       <dependencies>
         <dependency>
           <groupId>net.hydromatic</groupId>

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/DrillbitClassLoader.java
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/DrillbitClassLoader.java b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/DrillbitClassLoader.java
new file mode 100644
index 0000000..0ff3a99
--- /dev/null
+++ b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/DrillbitClassLoader.java
@@ -0,0 +1,106 @@
+/**
+ * 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.drill.jdbc;
+
+import java.io.File;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.util.ArrayList;
+import java.util.List;
+
+public class DrillbitClassLoader extends URLClassLoader {
+
+  public DrillbitClassLoader() {
+    super(URLS);
+  }
+
+  private static final URL[] URLS;
+
+  static {
+    ArrayList<URL> urlList = new ArrayList<URL>();
+    final String classPath = System.getProperty("app.class.path");
+    final String[] st = fracture(classPath, File.pathSeparator);
+    final int l = st.length;
+    for (int i = 0; i < l; i++) {
+      try {
+        if (st[i].length() == 0) {
+          st[i] = ".";
+        }
+        urlList.add(new File(st[i]).toURI().toURL());
+      } catch (MalformedURLException e) {
+        assert false : e.toString();
+      }
+    }
+    urlList.toArray(new URL[urlList.size()]);
+
+    List<URL> urls = new ArrayList<>();
+    for (URL url : urlList) {
+      urls.add(url);
+    }
+    URLS = urls.toArray(new URL[urls.size()]);
+  }
+
+  /**
+   * Helper method to avoid StringTokenizer using.
+   *
+   * Taken from Apache Harmony
+   */
+  private static String[] fracture(String str, String sep) {
+    if (str.length() == 0) {
+      return new String[0];
+    }
+    ArrayList<String> res = new ArrayList<String>();
+    int in = 0;
+    int curPos = 0;
+    int i = str.indexOf(sep);
+    int len = sep.length();
+    while (i != -1) {
+      String s = str.substring(curPos, i);
+      res.add(s);
+      in++;
+      curPos = i + len;
+      i = str.indexOf(sep, curPos);
+    }
+
+    len = str.length();
+    if (curPos <= len) {
+      String s = str.substring(curPos, len);
+      in++;
+      res.add(s);
+    }
+
+    return res.toArray(new String[in]);
+  }
+
+  @Override
+  protected Class<?> findClass(String name) throws ClassNotFoundException {
+    return super.findClass(name);
+  }
+
+  @Override
+  public Class<?> loadClass(String name) throws ClassNotFoundException {
+    return super.loadClass(name);
+  }
+
+  @Override
+  protected Class<?> loadClass(String name, boolean resolve) throws ClassNotFoundException {
+    return super.loadClass(name, resolve);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
new file mode 100644
index 0000000..97a0984
--- /dev/null
+++ b/exec/jdbc-all/src/test/java/org/apache/drill/jdbc/ITTestShadedJar.java
@@ -0,0 +1,189 @@
+/**
+ * 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.drill.jdbc;
+
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.net.MalformedURLException;
+import java.net.URL;
+import java.net.URLClassLoader;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+import java.util.Vector;
+
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class ITTestShadedJar {
+
+  private static DrillbitClassLoader drillbitLoader;
+  private static URLClassLoader rootClassLoader;
+
+  private static URL getJdbcUrl() throws MalformedURLException {
+    return new URL(
+        String.format("%s../../target/drill-jdbc-all-%s.jar",
+            ClassLoader.getSystemClassLoader().getResource("").toString(),
+            System.getProperty("project.version")
+            ));
+
+  }
+
+  @Test
+  public void executeJdbcAllQuery() throws Exception {
+
+    // print class path for debugging
+    System.out.println("java.class.path:");
+    System.out.println(System.getProperty("java.class.path"));
+
+    final URLClassLoader loader = (URLClassLoader) ClassLoader.getSystemClassLoader();
+    Method method = URLClassLoader.class.getDeclaredMethod("addURL", URL.class);
+    method.setAccessible(true);
+    method.invoke(loader, getJdbcUrl());
+
+    Class<?> clazz = loader.loadClass("org.apache.drill.jdbc.Driver");
+    try {
+      Driver driver = (Driver) clazz.newInstance();
+      try (Connection c = driver.connect("jdbc:drill:drillbit=localhost:31010", null);
+          Statement s = c.createStatement();
+          ResultSet result = s.executeQuery("select * from (VALUES 1)");) {
+        while (result.next()) {
+          System.out.println(result.getObject(1));
+        }
+      }
+    } catch (Exception ex) {
+      throw ex;
+    }
+
+  }
+
+
+
+  @BeforeClass
+  public static void setupDefaultTestCluster() throws Exception {
+    drillbitLoader = new DrillbitClassLoader();
+    rootClassLoader = (URLClassLoader) Thread.currentThread().getContextClassLoader();
+    try {
+    runWithLoader("DrillbitStartThread", drillbitLoader);
+    } catch (Exception e) {
+      printClassesLoaded("root", rootClassLoader);
+      throw e;
+    }
+  }
+
+  @AfterClass
+  public static void closeClient() throws Exception {
+    runWithLoader("DrillbitStopThread", drillbitLoader);
+  }
+
+  private static int getClassesLoadedCount(ClassLoader classLoader) {
+    try {
+      Field f = ClassLoader.class.getDeclaredField("classes");
+      f.setAccessible(true);
+      Vector<Class> classes = (Vector<Class>) f.get(classLoader);
+      return classes.size();
+    } catch (Exception e) {
+      System.out.println("Failure while loading class count.");
+      return -1;
+    }
+  }
+
+  private static void printClassesLoaded(String prefix, ClassLoader classLoader) {
+    try {
+      Field f = ClassLoader.class.getDeclaredField("classes");
+      f.setAccessible(true);
+      Vector<Class> classes = (Vector<Class>) f.get(classLoader);
+      for (Class<?> c : classes) {
+        System.out.println(prefix + ": " + c.getName());
+      }
+    } catch (Exception e) {
+      System.out.println("Failure while printing loaded classes.");
+    }
+  }
+
+  private static void runWithLoader(String name, ClassLoader loader) throws Exception {
+    Class<?> clazz = loader.loadClass(ITTestShadedJar.class.getName() + "$" + name);
+    Object o = clazz.getDeclaredConstructors()[0].newInstance(loader);
+    clazz.getMethod("go").invoke(o);
+  }
+
+  public abstract static class AbstractLoaderThread extends Thread {
+    private Exception ex;
+    protected final ClassLoader loader;
+
+    public AbstractLoaderThread(ClassLoader loader) {
+      this.setContextClassLoader(loader);
+      this.loader = loader;
+    }
+
+    @Override
+    public final void run() {
+      try {
+        internalRun();
+      } catch (Exception e) {
+        this.ex = e;
+      }
+    }
+
+    protected abstract void internalRun() throws Exception;
+
+    public void go() throws Exception {
+      start();
+      join();
+      if (ex != null) {
+        throw ex;
+      }
+    }
+  }
+
+  public static class DrillbitStartThread extends AbstractLoaderThread {
+
+    public DrillbitStartThread(ClassLoader loader) {
+      super(loader);
+    }
+
+    @Override
+    protected void internalRun() throws Exception {
+      Class<?> clazz = loader.loadClass("org.apache.drill.BaseTestQuery");
+      clazz.getMethod("setupDefaultTestCluster").invoke(null);
+
+      // loader.loadClass("org.apache.drill.exec.exception.SchemaChangeException");
+
+      // execute a single query to make sure the drillbit is fully up
+      clazz.getMethod("testNoResult", String.class, new Object[] {}.getClass())
+          .invoke(null, "select * from (VALUES 1)", new Object[] {});
+
+    }
+
+  }
+
+  public static class DrillbitStopThread extends AbstractLoaderThread {
+
+    public DrillbitStopThread(ClassLoader loader) {
+      super(loader);
+    }
+
+    @Override
+    protected void internalRun() throws Exception {
+      Class<?> clazz = loader.loadClass("org.apache.drill.BaseTestQuery");
+      clazz.getMethod("setupDefaultTestCluster").invoke(null);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/jdbc-all/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/src/test/resources/logback.xml b/exec/jdbc-all/src/test/resources/logback.xml
new file mode 100644
index 0000000..54ccb42
--- /dev/null
+++ b/exec/jdbc-all/src/test/resources/logback.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<configuration>
+  <appender name="SOCKET"
+    class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+    <Compressing>true</Compressing>
+    <ReconnectionDelay>10000</ReconnectionDelay>
+    <IncludeCallerData>true</IncludeCallerData>
+    <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+  </appender>
+
+  <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n
+      </pattern>
+    </encoder>
+  </appender>
+
+  <logger name="org.apache.drill" additivity="false">
+    <level value="debug" />
+    <appender-ref ref="SOCKET" />
+  </logger>
+
+  <logger name="query.logger" additivity="false">
+    <level value="info" />
+    <appender-ref ref="SOCKET" />
+  </logger>
+
+<!--   <logger name="io.netty" additivity="false"> -->
+<!--     <level value="debug" /> -->
+<!--     <appender-ref ref="SOCKET" /> -->
+<!--   </logger> -->
+
+  <root>
+    <level value="error" />
+    <appender-ref ref="STDOUT" />
+  </root>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/drill/blob/4e3b7dc0/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java b/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
index 1ef84c4..0d0300e 100644
--- a/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
+++ b/exec/jdbc/src/main/java/org/apache/drill/jdbc/Driver.java
@@ -26,15 +26,12 @@ import java.util.Properties;
 
 import org.apache.drill.jdbc.impl.DriverImpl;
 
-import org.slf4j.Logger;
-import static org.slf4j.LoggerFactory.getLogger;
-
 
 /**
  * Main class of Apache Drill JDBC driver.
  */
 public class Driver implements java.sql.Driver {
-  private static final Logger logger = getLogger( Driver.class );
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Driver.class);
 
   /** Delegate for everything except registration with DriverManager. */
   private final DriverImpl impl;
@@ -46,13 +43,10 @@ public class Driver implements java.sql.Driver {
 
   static {
     // Upon loading of class, register an instance with DriverManager.
-
     try {
-      DriverManager.registerDriver( new Driver() );
-    }
-    catch ( SQLException e ) {
-      logger.error( "Error in registering Drill JDBC driver {}: {}",
-                    Driver.class, e, e );
+      DriverManager.registerDriver(new Driver());
+    } catch (Error | SQLException e) {
+      logger.warn("Error in registering Drill JDBC driver {}: {}", Driver.class, e, e);
     }
   }
 


[15/15] drill git commit: Disable TestExampleQueries.testTextPartitions() until DRILL-3774 is fixed.

Posted by ja...@apache.org.
Disable TestExampleQueries.testTextPartitions() until DRILL-3774 is fixed.


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

Branch: refs/heads/master
Commit: e5f529b611d4c2a6f674e09c12078a5441b5ce10
Parents: 71086bf
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Sep 13 19:14:26 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:34 2015 -0700

----------------------------------------------------------------------
 .../src/test/java/org/apache/drill/TestExampleQueries.java    | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e5f529b6/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
index 58c7862..e88e2b3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestExampleQueries.java
@@ -17,8 +17,11 @@
  */
 package org.apache.drill;
 
+import static org.apache.drill.TestBuilder.listOf;
 import static org.junit.Assert.assertEquals;
 
+import java.math.BigDecimal;
+
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.common.util.TestTools;
@@ -26,9 +29,6 @@ import org.apache.drill.exec.ExecConstants;
 import org.junit.Ignore;
 import org.junit.Test;
 
-import java.math.BigDecimal;
-import static org.apache.drill.TestBuilder.listOf;
-
 public class TestExampleQueries extends BaseTestQuery {
 //  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExampleQueries.class);
 
@@ -278,6 +278,7 @@ public class TestExampleQueries extends BaseTestQuery {
   }
 
   @Test
+  @Ignore("DRILL-3774")
   public void testTextPartitions() throws Exception {
     String root = FileUtils.getResourceAsFile("/store/text/data/").toURI().toString();
     String query = String.format("select * from dfs_test.`%s`", root);


[12/15] drill git commit: DRILL-3160: Make JDBC Javadoc documentation available to users

Posted by ja...@apache.org.
DRILL-3160: Make JDBC Javadoc documentation available to users


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

Branch: refs/heads/master
Commit: e43155d8eabb6fc2d0fa4c68c25d6e7c59bf4521
Parents: 26477cf
Author: Aditya Kishore <ad...@apache.org>
Authored: Sun Sep 13 19:44:48 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:34 2015 -0700

----------------------------------------------------------------------
 exec/jdbc-all/pom.xml | 30 ++++++++++++++++++++++++++++++
 1 file changed, 30 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e43155d8/exec/jdbc-all/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc-all/pom.xml b/exec/jdbc-all/pom.xml
index 5ef6e07..d2cff3f 100644
--- a/exec/jdbc-all/pom.xml
+++ b/exec/jdbc-all/pom.xml
@@ -447,6 +447,36 @@
   </pluginRepositories>
 
   <profiles>
+    <profile>
+      <id>apache-release</id>
+      <build>
+        <plugins>
+          <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-javadoc-plugin</artifactId>
+            <executions>
+              <execution>
+                <id>attach-javadocs</id>
+                <goals>
+                  <goal>aggregate-jar</goal>
+                </goals>
+                <inherited>false</inherited>
+                <configuration>
+                  <!-- Only inlcude the published interface in the java-doc. Unfortunately,
+                       the javadoc tool and in turn, the plugin does not support inclusion
+                       pattern and hence we have to rely on exclusion pattern. -->
+                  <excludePackageNames>org.apache.drill.jdbc.impl</excludePackageNames>
+                  <includeDependencySources>true</includeDependencySources>
+                  <dependencySourceIncludes>
+                    <dependencySourceInclude>org.apache.drill.exec:drill-jdbc</dependencySourceInclude>
+                  </dependencySourceIncludes>
+                </configuration>
+              </execution>
+            </executions>
+          </plugin>
+        </plugins>
+      </build>
+    </profile>
     <!-- mondrian data includes 10s of MBs of JSON file if you want to include 
       them run maven with -Pwith-mondrian-data -->
     <profile>


[07/15] drill git commit: DRILL-3773: Fix Mongo FieldSelection

Posted by ja...@apache.org.
DRILL-3773: Fix Mongo FieldSelection

Mongo plugin was previously rewriting a complex (multi-level) column reference as a simple selection of the top level field.

This changeset does not change this behavior in terms of the filter sent to mongo, but it add the original selected column to the list that will be read in by the JSON reader once that data is returned from mongo.

What this means is that we will be requesting more data from mongo that necessary (as we were previously), but this will be leveraging the existing functionality in the JSON reader to grab only the sub-selection actually requested in the query. This allows for difficult schema changes to be avoided by projecting only columns without schema changes.

This also fixes and adds unit tests for FieldSelection that cause wrong results when selecting a nested column and its parent.


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

Branch: refs/heads/master
Commit: 97615e5675c1b25a4a9b5f96e6e1be7ed4f96c9c
Parents: 197d972
Author: Jason Altekruse <al...@gmail.com>
Authored: Wed Sep 9 11:26:09 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 18:36:35 2015 -0700

----------------------------------------------------------------------
 .../exec/store/mongo/MongoRecordReader.java     |  2 +-
 .../exec/store/mongo/MongoTestConstants.java    |  2 +
 .../drill/exec/store/mongo/MongoTestSuit.java   |  2 +
 .../store/mongo/TestMongoProjectPushDown.java   | 43 ++++++++++++++++++++
 .../resources/schema_change_int_to_string.json  | 31 ++++++++++++++
 .../exec/vector/complex/fn/FieldSelection.java  |  2 +-
 .../java/org/apache/drill/DrillTestWrapper.java | 20 ++++++++-
 .../vector/complex/writer/TestJsonReader.java   | 31 ++++++++++++++
 8 files changed, 129 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
index 0ac519f..c8b0699 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/MongoRecordReader.java
@@ -98,7 +98,7 @@ public class MongoRecordReader extends AbstractRecordReader {
     if (!isStarQuery()) {
       for (SchemaPath column : projectedColumns ) {
         String fieldName = column.getRootSegment().getPath();
-        transformed.add(SchemaPath.getSimplePath(fieldName));
+        transformed.add(column);
         this.fields.put(fieldName, Integer.valueOf(1));
       }
     } else {

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
index d6d83d1..d050961 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
@@ -39,9 +39,11 @@ public interface MongoTestConstants {
 
   public static final String DONUTS_COLLECTION = "donuts";
   public static final String EMPINFO_COLLECTION = "empinfo";
+  public static final String SCHEMA_CHANGE_COLLECTION = "schema_change";
 
   public static final String DONUTS_DATA = "donuts.json";
   public static final String EMP_DATA = "emp.json";
+  public static final String SCHEMA_CHANGE_DATA = "schema_change_int_to_string.json";
 
   public static final String REPLICA_SET_1_NAME = "shard_1_replicas";
   public static final String REPLICA_SET_2_NAME = "shard_2_replicas";

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
index c52df63..284e7d8 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
@@ -186,6 +186,7 @@ public class MongoTestSuit implements MongoTestConstants {
       mongod = mongodExecutable.start();
       mongoClient = new MongoClient(new ServerAddress(LOCALHOST, MONGOS_PORT));
       createDbAndCollections(EMPLOYEE_DB, EMPINFO_COLLECTION, "employee_id");
+      createDbAndCollections(EMPLOYEE_DB, SCHEMA_CHANGE_COLLECTION, "field_2");
     }
 
     private static void cleanup() {
@@ -209,6 +210,7 @@ public class MongoTestSuit implements MongoTestConstants {
         SingleMode.setup();
       }
       TestTableGenerator.importData(EMPLOYEE_DB, EMPINFO_COLLECTION, EMP_DATA);
+      TestTableGenerator.importData(EMPLOYEE_DB, SCHEMA_CHANGE_COLLECTION, SCHEMA_CHANGE_DATA);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
index 54ace3f..b17cf2f 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
@@ -17,10 +17,53 @@
  */
 package org.apache.drill.exec.store.mongo;
 
+import org.apache.drill.exec.ExecConstants;
+import org.junit.BeforeClass;
 import org.junit.Test;
 
+import java.io.IOException;
+
+import static org.apache.drill.TestBuilder.listOf;
+import static org.apache.drill.TestBuilder.mapOf;
+
 public class TestMongoProjectPushDown extends MongoTestBase {
 
+  /**
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testComplexProjectPushdown() throws Exception {
+
+    try {
+      testBuilder()
+          .sqlQuery("select t.field_4.inner_3 as col_1, t.field_4 as col_2 from mongo.employee.schema_change t")
+          .unOrdered()
+          .optionSettingQueriesForTestQuery(String.format("alter session set `%s` = true", ExecConstants.MONGO_READER_READ_NUMBERS_AS_DOUBLE))
+              .baselineColumns("col_1", "col_2")
+              .baselineValues(
+                  mapOf(),
+                  mapOf(
+                      "inner_1", listOf(),
+                      "inner_3", mapOf()))
+              .baselineValues(
+                  mapOf("inner_object_field_1", 2.0),
+                  mapOf(
+                      "inner_1", listOf(1.0, 2.0, 3.0),
+                      "inner_2", 3.0,
+                      "inner_3", mapOf("inner_object_field_1", 2.0)))
+              .baselineValues(
+                  mapOf(),
+                  mapOf(
+                      "inner_1", listOf(4.0, 5.0, 6.0),
+                      "inner_2", 3.0,
+                      "inner_3", mapOf()))
+              .go();
+    } finally {
+      test(String.format("alter session set `%s` = false", ExecConstants.MONGO_READER_READ_NUMBERS_AS_DOUBLE));
+    }
+  }
+
   @Test
   public void testSingleColumnProject() throws Exception {
     String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1,

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/contrib/storage-mongo/src/test/resources/schema_change_int_to_string.json
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/resources/schema_change_int_to_string.json b/contrib/storage-mongo/src/test/resources/schema_change_int_to_string.json
new file mode 100644
index 0000000..d6ab4c9
--- /dev/null
+++ b/contrib/storage-mongo/src/test/resources/schema_change_int_to_string.json
@@ -0,0 +1,31 @@
+[{
+  "field_1": [1]
+},
+{
+  "field_1": [5],
+  "field_2": 2,
+  "field_3": {
+  "inner_1" : 2
+},
+  "field_4" : {
+  "inner_1" : [1,2,3],
+  "inner_2" : 3,
+  "inner_3" :  { "inner_object_field_1" : 2}
+},
+  "field_5" : [ { "inner_list" : [1, null, 6] }, { "inner_list":[3,8]}, { "inner_list":[12, null, 4, "null", 5]} ]
+},
+{
+  "field_1": [5,10,15],
+  "field_2": "A wild string appears!",
+  "field_3": {
+    "inner_1" : 5,
+    "inner_2" : 3,
+    "inner_3" : [ { "inner_object_field_1" : null}, {"inner_object_field_1" : 10} ]
+  },
+  "field_4" : {
+    "inner_1" : [4,5,6],
+    "inner_2" : 3
+  },
+  "field_5" : [ { "inner_list" : [5, null, 6.0, "1234"] }, { "inner_list":[7,8.0, "12341324"], "inner_list_2" : [1,2,2323.443e10, "hello there"]}, { "inner_list":[3,4,5], "inner_list_2" : [10, 11, 12]} ]
+}
+]

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/FieldSelection.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/FieldSelection.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/FieldSelection.java
index aecff05..1857479 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/FieldSelection.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/FieldSelection.java
@@ -70,7 +70,7 @@ class FieldSelection {
       for(Entry<String, FieldSelection> e : children.entrySet()){
         newMap.put(e.getKey(), e.getValue().fixNodes());
       }
-      return new FieldSelection(newMap, ValidityMode.CHECK_CHILDREN);
+      return new FieldSelection(newMap, mode);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java b/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
index ff75274..59cbb15 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/DrillTestWrapper.java
@@ -397,7 +397,7 @@ public class DrillTestWrapper {
     if (expectedNumBatches != EXPECTED_BATCH_COUNT_NOT_SET) {
       final int actualNumBatches = results.size();
       assertEquals(String.format("Expected %d batches but query returned %d non empty batch(es)%n", expectedNumBatches,
-        actualNumBatches), expectedNumBatches, actualNumBatches);
+          actualNumBatches), expectedNumBatches, actualNumBatches);
     }
   }
 
@@ -560,7 +560,23 @@ public class DrillTestWrapper {
         break;
       }
       if (!found) {
-        throw new Exception(String.format("After matching %d records, did not find expected record in result set: %s", counter, printRecord(expectedRecord)));
+        StringBuilder sb = new StringBuilder();
+        for (int expectedRecordDisplayCount = 0;
+             expectedRecordDisplayCount < 10 && expectedRecordDisplayCount < expectedRecords.size();
+             expectedRecordDisplayCount++) {
+          sb.append(printRecord(expectedRecords.get(expectedRecordDisplayCount)));
+        }
+        String expectedRecordExamples = sb.toString();
+        sb.setLength(0);
+        for (int actualRecordDisplayCount = 0;
+             actualRecordDisplayCount < 10 && actualRecordDisplayCount < actualRecords.size();
+             actualRecordDisplayCount++) {
+          sb.append(printRecord(actualRecords.get(actualRecordDisplayCount)));
+        }
+        String actualRecordExamples = sb.toString();
+        throw new Exception(String.format("After matching %d records, did not find expected record in result set: %s\n\n" +
+            "Some examples of expected records:%s\n\n Some examples of records returned by the test query:%s",
+            counter, printRecord(expectedRecord), expectedRecordExamples, actualRecordExamples));
       } else {
         actualRecords.remove(i);
         counter++;

http://git-wip-us.apache.org/repos/asf/drill/blob/97615e56/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
index 7d6c71c..bd9cea1 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestJsonReader.java
@@ -63,6 +63,37 @@ public class TestJsonReader extends BaseTestQuery {
   }
 
   @Test
+  public void testFieldSelectionBug() throws Exception {
+    try {
+      testBuilder()
+          .sqlQuery("select t.field_4.inner_3 as col_1, t.field_4 as col_2 from cp.`store/json/schema_change_int_to_string.json` t")
+          .unOrdered()
+          .optionSettingQueriesForTestQuery("alter session set `store.json.all_text_mode` = true")
+          .baselineColumns("col_1", "col_2")
+          .baselineValues(
+              mapOf(),
+              mapOf(
+                  "inner_1", listOf(),
+                  "inner_3", mapOf()))
+          .baselineValues(
+              mapOf("inner_object_field_1", "2"),
+              mapOf(
+                  "inner_1", listOf("1", "2", "3"),
+                  "inner_2", "3",
+                  "inner_3", mapOf("inner_object_field_1", "2")))
+          .baselineValues(
+              mapOf(),
+              mapOf(
+                  "inner_1", listOf("4", "5", "6"),
+                  "inner_2", "3",
+                  "inner_3", mapOf()))
+          .go();
+    } finally {
+      test("alter session set `store.json.all_text_mode` = false");
+    }
+  }
+
+  @Test
   public void testSplitAndTransferFailure() throws Exception {
     final String testVal = "a string";
     testBuilder()


[14/15] drill git commit: Delete deprecated Jackson access proxies.

Posted by ja...@apache.org.
Delete deprecated Jackson access proxies.


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

Branch: refs/heads/master
Commit: 26477cfd650251ee2fa522986955eca071e0fcc1
Parents: 4e3b7dc
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Sep 13 19:40:30 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:34 2015 -0700

----------------------------------------------------------------------
 .../jackson/core/JsonStreamContextExposer.java  | 29 ---------------
 .../core/json/JsonReadContextExposer.java       | 39 --------------------
 2 files changed, 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/26477cfd/exec/java-exec/src/main/java/com/fasterxml/jackson/core/JsonStreamContextExposer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/com/fasterxml/jackson/core/JsonStreamContextExposer.java b/exec/java-exec/src/main/java/com/fasterxml/jackson/core/JsonStreamContextExposer.java
deleted file mode 100644
index 5099c79..0000000
--- a/exec/java-exec/src/main/java/com/fasterxml/jackson/core/JsonStreamContextExposer.java
+++ /dev/null
@@ -1,29 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.fasterxml.jackson.core;
-
-/**
- * Exposes certain package protected methods in Jackson.
- */
-public class JsonStreamContextExposer {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonStreamContextExposer.class);
-
-  public static int getType(JsonStreamContext c){
-    return c._type;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/26477cfd/exec/java-exec/src/main/java/com/fasterxml/jackson/core/json/JsonReadContextExposer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/com/fasterxml/jackson/core/json/JsonReadContextExposer.java b/exec/java-exec/src/main/java/com/fasterxml/jackson/core/json/JsonReadContextExposer.java
deleted file mode 100644
index 820dec8..0000000
--- a/exec/java-exec/src/main/java/com/fasterxml/jackson/core/json/JsonReadContextExposer.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.fasterxml.jackson.core.json;
-
-/**
- * Exposes certain package protected methods in Jackson.
- */
-public class JsonReadContextExposer {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JsonReadContextExposer.class);
-
-  public static int getColNmbr(JsonReadContext c){
-    return c._columnNr;
-  }
-
-  public static int getLineNmbr(JsonReadContext c){
-    return c._lineNr;
-  }
-
-
-  public static void reset(JsonReadContext c, int type, int lineNr, int colNr){
-    c.reset(type, lineNr, colNr);
-  }
-
-}


[08/15] drill git commit: DRILL-2879: Enhancing extended json support for date in millies and binary with type info

Posted by ja...@apache.org.
DRILL-2879: Enhancing extended json support for date in millies and binary with type info

Ignore project push down Mongo test until test completes correctly on Linux


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

Branch: refs/heads/master
Commit: 221489406505d738f1746dbf49f8dd0d67df3fc2
Parents: 97615e5
Author: Kamesh <ka...@gmail.com>
Authored: Wed Jul 15 16:34:41 2015 +0530
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:07 2015 -0700

----------------------------------------------------------------------
 .../store/mongo/TestMongoProjectPushDown.java   | 11 ++-
 .../exec/store/easy/json/JSONRecordReader.java  |  7 +-
 .../exec/vector/complex/fn/ExtendedType.java    |  1 +
 .../vector/complex/fn/ExtendedTypeName.java     |  1 +
 .../exec/vector/complex/fn/VectorOutput.java    | 81 +++++++++++++++++---
 .../complex/writer/TestExtendedTypes.java       | 26 +++++++
 .../vector/complex/mongo_extended.json          |  8 ++
 7 files changed, 115 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
index b17cf2f..32666fc 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
@@ -17,15 +17,14 @@
  */
 package org.apache.drill.exec.store.mongo;
 
-import org.apache.drill.exec.ExecConstants;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.io.IOException;
-
 import static org.apache.drill.TestBuilder.listOf;
 import static org.apache.drill.TestBuilder.mapOf;
 
+import org.apache.drill.exec.ExecConstants;
+import org.junit.Ignore;
+import org.junit.Test;
+
+@Ignore("DRILL-3775")
 public class TestMongoProjectPushDown extends MongoTestBase {
 
   /**

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
index dfc4f3a..8e78cf1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONRecordReader.java
@@ -155,8 +155,11 @@ public class JSONRecordReader extends AbstractRecordReader {
     if (columnNr > 0) {
       exceptionBuilder.pushContext("Column ", columnNr);
     }
-    exceptionBuilder.pushContext("Record ", currentRecordNumberInFile())
-            .pushContext("File ", hadoopPath.toUri().getPath());
+
+    if (hadoopPath != null) {
+      exceptionBuilder.pushContext("Record ", currentRecordNumberInFile())
+          .pushContext("File ", hadoopPath.toUri().getPath());
+    }
 
     throw exceptionBuilder.build(logger);
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
index bec0fd2..13df44f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedType.java
@@ -22,6 +22,7 @@ import com.fasterxml.jackson.core.io.SerializedString;
 public enum ExtendedType {
 
   BINARY(ExtendedTypeName.BINARY),
+  TYPE(ExtendedTypeName.TYPE),
   DATE(ExtendedTypeName.DATE),
   TIME(ExtendedTypeName.TIME),
   TIMESTAMP(ExtendedTypeName.TIMESTAMP),

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
index fcef24b..e432d56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/ExtendedTypeName.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.vector.complex.fn;
 
 public interface ExtendedTypeName {
   public static final String BINARY = "$binary";      // base64 encoded binary (ZHJpbGw=)  [from Mongo]
+  public static final String TYPE = "$type";          // type of binary data
   public static final String DATE = "$dateDay";       // ISO date with no time. such as (12-24-27)
   public static final String TIME = "$time";          // ISO time with no timezone (19:20:30.45Z)
   public static final String TIMESTAMP = "$date";     // ISO standard time (2009-02-23T00:00:00.000-08:00) [from Mongo]

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
index 651de3d..769f341 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/complex/fn/VectorOutput.java
@@ -81,7 +81,7 @@ abstract class VectorOutput {
       switch(possibleTypeName){
       case ExtendedTypeName.BINARY:
         writeBinary(checkNextToken(JsonToken.VALUE_STRING));
-        checkNextToken(JsonToken.END_OBJECT);
+        checkCurrentToken(JsonToken.END_OBJECT);
         return true;
       case ExtendedTypeName.DATE:
         writeDate(checkNextToken(JsonToken.VALUE_STRING));
@@ -92,7 +92,7 @@ abstract class VectorOutput {
         checkNextToken(JsonToken.END_OBJECT);
         return true;
       case ExtendedTypeName.TIMESTAMP:
-        writeTimestamp(checkNextToken(JsonToken.VALUE_STRING));
+        writeTimestamp(checkNextToken(JsonToken.VALUE_STRING, JsonToken.VALUE_NUMBER_INT));
         checkNextToken(JsonToken.END_OBJECT);
         return true;
       case ExtendedTypeName.INTERVAL:
@@ -100,7 +100,7 @@ abstract class VectorOutput {
         checkNextToken(JsonToken.END_OBJECT);
         return true;
       case ExtendedTypeName.INTEGER:
-        writeInteger(checkNextToken(JsonToken.VALUE_NUMBER_INT));
+        writeInteger(checkNextToken(JsonToken.VALUE_STRING, JsonToken.VALUE_NUMBER_INT));
         checkNextToken(JsonToken.END_OBJECT);
         return true;
       case ExtendedTypeName.DECIMAL:
@@ -116,8 +116,35 @@ abstract class VectorOutput {
     return checkNextToken(expected, expected);
   }
 
+  public boolean checkCurrentToken(final JsonToken expected) throws IOException{
+    return checkCurrentToken(expected, expected);
+  }
+
   public boolean checkNextToken(final JsonToken expected1, final JsonToken expected2) throws IOException{
-    JsonToken t = parser.nextToken();
+    return checkToken(parser.nextToken(), expected1, expected2);
+  }
+
+  public boolean checkCurrentToken(final JsonToken expected1, final JsonToken expected2) throws IOException{
+    return checkToken(parser.getCurrentToken(), expected1, expected2);
+  }
+
+  boolean hasType() throws JsonParseException, IOException {
+    JsonToken token = parser.nextToken();
+    return token == JsonToken.FIELD_NAME && parser.getText().equals(ExtendedTypeName.TYPE);
+  }
+
+  long getType() throws JsonParseException, IOException {
+    if (!checkNextToken(JsonToken.VALUE_NUMBER_INT, JsonToken.VALUE_STRING)) {
+      long type = parser.getValueAsLong();
+      //Advancing the token, as checking current token in binary
+      parser.nextToken();
+      return type;
+    }
+    throw new JsonParseException("Failure while reading $type value. Expected a NUMBER or STRING",
+        parser.getCurrentLocation());
+  }
+
+  public boolean checkToken(final JsonToken t, final JsonToken expected1, final JsonToken expected2) throws IOException{
     if(t == JsonToken.VALUE_NULL){
       return true;
     }else if(t == expected1){
@@ -154,7 +181,12 @@ abstract class VectorOutput {
     public void writeBinary(boolean isNull) throws IOException {
       VarBinaryWriter bin = writer.varBinary();
       if(!isNull){
-        work.prepareBinary(parser.getBinaryValue(), binary);
+        byte[] binaryData = parser.getBinaryValue();
+        if (hasType()) {
+          //Ignoring type info as of now.
+          getType();
+        }
+        work.prepareBinary(binaryData, binary);
         bin.write(binary);
       }
     }
@@ -181,8 +213,18 @@ abstract class VectorOutput {
     public void writeTimestamp(boolean isNull) throws IOException {
       TimeStampWriter ts = writer.timeStamp();
       if(!isNull){
-        DateTimeFormatter f = ISODateTimeFormat.dateTime();
-        ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+        switch (parser.getCurrentToken()) {
+        case VALUE_NUMBER_INT:
+          DateTime dt = new DateTime(parser.getLongValue(), org.joda.time.DateTimeZone.UTC);
+          ts.writeTimeStamp(dt.getMillis());
+          break;
+        case VALUE_STRING:
+          DateTimeFormatter f = ISODateTimeFormat.dateTime();
+          ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+          break;
+        default:
+          break;
+        }
       }
     }
 
@@ -202,7 +244,7 @@ abstract class VectorOutput {
     public void writeInteger(boolean isNull) throws IOException {
       BigIntWriter intWriter = writer.bigInt();
       if(!isNull){
-        intWriter.writeBigInt(parser.getLongValue());
+        intWriter.writeBigInt(Long.parseLong(parser.getValueAsString()));
       }
     }
 
@@ -232,7 +274,12 @@ abstract class VectorOutput {
     public void writeBinary(boolean isNull) throws IOException {
       VarBinaryWriter bin = writer.varBinary(fieldName);
       if(!isNull){
-        work.prepareBinary(parser.getBinaryValue(), binary);
+        byte[] binaryData = parser.getBinaryValue();
+        if (hasType()) {
+          //Ignoring type info as of now.
+          getType();
+        }
+        work.prepareBinary(binaryData, binary);
         bin.write(binary);
       }
     }
@@ -260,8 +307,18 @@ abstract class VectorOutput {
     public void writeTimestamp(boolean isNull) throws IOException {
       TimeStampWriter ts = writer.timeStamp(fieldName);
       if(!isNull){
-        DateTimeFormatter f = ISODateTimeFormat.dateTime();
-        ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+        switch (parser.getCurrentToken()) {
+        case VALUE_NUMBER_INT:
+          DateTime dt = new DateTime(parser.getLongValue(), org.joda.time.DateTimeZone.UTC);
+          ts.writeTimeStamp(dt.getMillis());
+          break;
+        case VALUE_STRING:
+          DateTimeFormatter f = ISODateTimeFormat.dateTime();
+          ts.writeTimeStamp(DateTime.parse(parser.getValueAsString(), f).withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis());
+          break;
+        default:
+          break;
+        }
       }
     }
 
@@ -281,7 +338,7 @@ abstract class VectorOutput {
     public void writeInteger(boolean isNull) throws IOException {
       BigIntWriter intWriter = writer.bigInt(fieldName);
       if(!isNull){
-        intWriter.writeBigInt(parser.getLongValue());
+        intWriter.writeBigInt(Long.parseLong(parser.getValueAsString()));
       }
     }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
index f403108..51ecec5 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestExtendedTypes.java
@@ -64,4 +64,30 @@ public class TestExtendedTypes extends BaseTestQuery {
           ExecConstants.JSON_EXTENDED_TYPES.getDefault().getValue()));
     }
   }
+
+  @Test
+  public void testMongoExtendedTypes() throws Exception {
+
+    final String originalFile = "${WORKING_PATH}/src/test/resources/vector/complex/mongo_extended.json".replaceAll(
+        Pattern.quote("${WORKING_PATH}"),
+        Matcher.quoteReplacement(TestTools.getWorkingPath()));
+
+    try {
+      testNoResult(String.format("ALTER SESSION SET `%s` = 'json'", ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName()));
+      testNoResult(String.format("ALTER SESSION SET `%s` = true", ExecConstants.JSON_EXTENDED_TYPES.getOptionName()));
+
+      int actualRecordCount = testSql(String.format("select * from dfs.`%s`", originalFile));
+      assertEquals(
+          String.format(
+              "Received unexpected number of rows in output: expected=%d, received=%s",
+              1, actualRecordCount), 1, actualRecordCount);
+    } finally {
+      testNoResult(String.format("ALTER SESSION SET `%s` = '%s'",
+          ExecConstants.OUTPUT_FORMAT_VALIDATOR.getOptionName(),
+          ExecConstants.OUTPUT_FORMAT_VALIDATOR.getDefault().getValue()));
+      testNoResult(String.format("ALTER SESSION SET `%s` = %s",
+          ExecConstants.JSON_EXTENDED_TYPES.getOptionName(),
+          ExecConstants.JSON_EXTENDED_TYPES.getDefault().getValue()));
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/22148940/exec/java-exec/src/test/resources/vector/complex/mongo_extended.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/vector/complex/mongo_extended.json b/exec/java-exec/src/test/resources/vector/complex/mongo_extended.json
new file mode 100644
index 0000000..a38a83f
--- /dev/null
+++ b/exec/java-exec/src/test/resources/vector/complex/mongo_extended.json
@@ -0,0 +1,8 @@
+{
+  "drill_timestamp_millies" : {
+    "$date" : 1436241583488
+  },
+  "bin" : {
+      "$binary" : "ZHJpbGw=", "$type" : 1
+  }
+}


[10/15] drill git commit: DRILL-3458: Enhancing Avro file format by support nested data types like union, map

Posted by ja...@apache.org.
DRILL-3458: Enhancing Avro file format by support nested data types like union, map

Removing unused imports and fixing typos in the comments

Adding sanity check to verify whether union is an optional field or not and negative testcases

Handling string and utf-8 datatypes properly

removing bracket notation and documentation


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

Branch: refs/heads/master
Commit: fe07b6c47682b898b60a9d8221e10207240061ff
Parents: 8f4ca6e
Author: Kamesh <ka...@gmail.com>
Authored: Sat Jul 4 23:35:48 2015 +0530
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:33 2015 -0700

----------------------------------------------------------------------
 .../drill/exec/store/avro/AvroRecordReader.java |  65 +++-
 .../drill/exec/store/avro/MapOrListWriter.java  |   9 +
 .../drill/exec/store/avro/AvroFormatTest.java   |  77 ++++-
 .../drill/exec/store/avro/AvroTestUtil.java     | 306 +++++++++++++++++++
 4 files changed, 438 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/fe07b6c4/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
index 271c8e9..a09cd53 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
@@ -22,10 +22,13 @@ import io.netty.buffer.DrillBuf;
 import java.io.IOException;
 import java.io.UnsupportedEncodingException;
 import java.nio.ByteBuffer;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map.Entry;
 import java.util.concurrent.TimeUnit;
 
 import org.apache.avro.Schema;
+import org.apache.avro.Schema.Type;
 import org.apache.avro.file.DataFileReader;
 import org.apache.avro.generic.GenericArray;
 import org.apache.avro.generic.GenericContainer;
@@ -53,6 +56,7 @@ import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 
+import com.google.common.base.Charsets;
 import com.google.common.base.Stopwatch;
 
 /**
@@ -164,17 +168,22 @@ public class AvroRecordReader extends AbstractRecordReader {
     }
   }
 
-  private void process(final Object value, final Schema schema, final String fieldName, final MapOrListWriter writer) {
-
-    writer.start();
+  private void process(final Object value, final Schema schema, final String fieldName, MapOrListWriter writer) {
+    if (value == null) {
+      return;
+    }
     final Schema.Type type = schema.getType();
 
     switch (type) {
       case RECORD:
-        for (final Schema.Field field : schema.getFields()) {
+        // list field of MapOrListWriter will be non null when we want to store array of maps/records.
+        MapOrListWriter _writer = writer;
 
-          MapOrListWriter _writer = writer;
-          if (field.schema().getType() == Schema.Type.RECORD) {
+        for (final Schema.Field field : schema.getFields()) {
+          if (field.schema().getType() == Schema.Type.RECORD ||
+              (field.schema().getType() == Schema.Type.UNION &&
+              field.schema().getTypes().get(0).getType() == Schema.Type.NULL &&
+              field.schema().getTypes().get(1).getType() == Schema.Type.RECORD)) {
             _writer = writer.map(field.name());
           }
 
@@ -184,13 +193,38 @@ public class AvroRecordReader extends AbstractRecordReader {
       case ARRAY:
         assert fieldName != null;
         final GenericArray array = (GenericArray) value;
+        Schema elementSchema = array.getSchema().getElementType();
+        Type elementType = elementSchema.getType();
+        if (elementType == Schema.Type.RECORD || elementType == Schema.Type.MAP){
+          writer = writer.list(fieldName).listoftmap(fieldName);
+        } else {
+          writer = writer.list(fieldName);
+        }
+        writer.start();
         for (final Object o : array) {
-          process(o, array.getSchema().getElementType(), fieldName, writer.list(fieldName));
+          process(o, elementSchema, fieldName, writer);
         }
+        writer.end();
         break;
-      case FIXED:
       case UNION:
+        // currently supporting only nullable union (optional fields) like ["null", "some-type"].
+        if (schema.getTypes().get(0).getType() != Schema.Type.NULL) {
+          throw new UnsupportedOperationException("Avro union type must be of the format : [\"null\", \"some-type\"]");
+        }
+        process(value, schema.getTypes().get(1), fieldName, writer);
+        break;
       case MAP:
+        @SuppressWarnings("unchecked")
+        final HashMap<Object, Object> map = (HashMap<Object, Object>) value;
+        Schema valueSchema = schema.getValueType();
+        writer = writer.map(fieldName);
+        writer.start();
+        for (Entry<Object, Object> entry : map.entrySet()) {
+          process(entry.getValue(), valueSchema, entry.getKey().toString(), writer);
+        }
+        writer.end();
+        break;
+      case FIXED:
         throw new UnsupportedOperationException("Unimplemented type: " + type.toString());
       case ENUM:  // Enum symbols are strings
       case NULL:  // Treat null type as a primitive
@@ -214,19 +248,26 @@ public class AvroRecordReader extends AbstractRecordReader {
         break;
     }
 
-    writer.end();
   }
 
   private void processPrimitive(final Object value, final Schema.Type type, final String fieldName,
                                 final MapOrListWriter writer) {
+    if (value == null) {
+      return;
+    }
 
     switch (type) {
       case STRING:
-        final Utf8 utf8 = (Utf8) value;
-        final int length = utf8.length();
+        byte[] binary = null;
+        if (value instanceof Utf8) {
+          binary = ((Utf8) value).getBytes();
+        } else {
+          binary = value.toString().getBytes(Charsets.UTF_8);
+        }
+        final int length = binary.length;
         final VarCharHolder vh = new VarCharHolder();
         ensure(length);
-        buffer.setBytes(0, utf8.getBytes());
+        buffer.setBytes(0, binary);
         vh.buffer = buffer;
         vh.start = 0;
         vh.end = length;

http://git-wip-us.apache.org/repos/asf/drill/blob/fe07b6c4/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/MapOrListWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/MapOrListWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/MapOrListWriter.java
index d2a1031..1a94452 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/MapOrListWriter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/MapOrListWriter.java
@@ -66,6 +66,11 @@ public class MapOrListWriter {
     return new MapOrListWriter(map.map(name));
   }
 
+  MapOrListWriter listoftmap(final String name) {
+    assert list != null;
+    return new MapOrListWriter(list.map());
+  }
+
   MapOrListWriter list(final String name) {
     assert map != null;
     return new MapOrListWriter(map.list(name));
@@ -75,6 +80,10 @@ public class MapOrListWriter {
     return map != null;
   }
 
+  boolean isListWriter() {
+    return list != null;
+  }
+
   VarCharWriter varChar(final String name) {
     return (map != null) ? map.varChar(name) : list.varChar();
   }

http://git-wip-us.apache.org/repos/asf/drill/blob/fe07b6c4/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
index 2d2522b..bcbc707 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroFormatTest.java
@@ -18,7 +18,8 @@
 package org.apache.drill.exec.store.avro;
 
 import org.apache.drill.BaseTestQuery;
-
+import org.apache.drill.common.exceptions.UserRemoteException;
+import org.junit.Assert;
 import org.junit.Test;
 
 /**
@@ -79,8 +80,8 @@ public class AvroFormatTest extends BaseTestQuery {
   public void testSimpleNestedSchema_NoNullValues() throws Exception {
 
     final String file = AvroTestUtil.generateSimpleNestedSchema_NoNullValues();
-    final String sql = "select a_string, b_int, c_record['nested_1_string'], c_record['nested_1_int'] " +
-            "from dfs_test.`" + file + "`";
+    final String sql = "select a_string, b_int, t.c_record.nested_1_string, t.c_record.nested_1_int " +
+            "from dfs_test.`" + file + "` t";
     test(sql);
   }
 
@@ -96,10 +97,10 @@ public class AvroFormatTest extends BaseTestQuery {
   public void testDoubleNestedSchema_NoNullValues() throws Exception {
 
     final String file = AvroTestUtil.generateDoubleNestedSchema_NoNullValues();
-    final String sql = "select a_string, b_int, c_record['nested_1_string'], c_record['nested_1_int'], " +
-            "c_record['nested_1_record']['double_nested_1_string'], " +
-            "c_record['nested_1_record']['double_nested_1_int'] " +
-            "from dfs_test.`" + file + "`";
+    final String sql = "select a_string, b_int, t.c_record.nested_1_string, t.c_record.nested_1_int, " +
+            "t.c_record.nested_1_record.double_nested_1_string, " +
+            "t.c_record.nested_1_record.double_nested_1_int " +
+            "from dfs_test.`" + file + "` t";
     test(sql);
   }
 
@@ -126,4 +127,66 @@ public class AvroFormatTest extends BaseTestQuery {
     final String sql = "select * from dfs_test.`" + file + "`";
     test(sql);
   }
+
+  @Test
+  public void testSimpleUnionSchema_StarQuery() throws Exception {
+
+    final String file = AvroTestUtil.generateUnionSchema_WithNullValues();
+    final String sql = "select * from dfs_test.`" + file + "`";
+    test(sql);
+  }
+
+  @Test
+  public void testShouldFailSimpleUnionNonNullSchema_StarQuery() throws Exception {
+
+    final String file = AvroTestUtil.generateUnionSchema_WithNonNullValues();
+    final String sql = "select * from dfs_test.`" + file + "`";
+    try {
+      test(sql);
+      Assert.fail("Test should fail as union is only supported for optional fields");
+    } catch(UserRemoteException e) {
+      String message = e.getMessage();
+      Assert.assertTrue(message.contains("Avro union type must be of the format : [\"null\", \"some-type\"]"));
+    }
+  }
+
+  @Test
+  public void testNestedUnionSchema_withNullValues() throws Exception {
+
+    final String file = AvroTestUtil.generateUnionNestedSchema_withNullValues();
+    final String sql = "select t.c_record.nested_1_string,t.c_record.nested_1_int from dfs_test.`" + file + "` t";
+    test(sql);
+  }
+
+  @Test
+  public void testNestedUnionArraySchema_withNullValues() throws Exception {
+
+    final String file = AvroTestUtil.generateUnionNestedArraySchema_withNullValues();
+    final String sql = "select t.c_array[0].nested_1_string,t.c_array[0].nested_1_int from dfs_test.`" + file + "` t";
+    test(sql);
+  }
+
+  @Test
+  public void testMapSchema_withNullValues() throws Exception {
+
+    final String file = AvroTestUtil.generateMapSchema_withNullValues();
+    final String sql = "select c_map['key1'],c_map['key2'] from dfs_test.`" + file + "`";
+    test(sql);
+  }
+
+  @Test
+  public void testMapSchemaComplex_withNullValues() throws Exception {
+
+    final String file = AvroTestUtil.generateMapSchemaComplex_withNullValues();
+    final String sql = "select d_map['key1'],d_map['key2'] from dfs_test.`" + file + "`";
+    test(sql);
+  }
+
+  @Test
+  public void testStringAndUtf8Data() throws Exception {
+
+    final String file = AvroTestUtil.generateStringAndUtf8Data();
+    final String sql = "select * from dfs_test.`" + file + "`";
+    test(sql);
+  }
 }

http://git-wip-us.apache.org/repos/asf/drill/blob/fe07b6c4/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
index 419c054..d847b1a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/avro/AvroTestUtil.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.store.avro;
 
 import org.apache.avro.Schema;
+import org.apache.avro.Schema.Type;
 import org.apache.avro.SchemaBuilder;
 import org.apache.avro.file.DataFileWriter;
 import org.apache.avro.generic.GenericArray;
@@ -27,6 +28,8 @@ import org.apache.avro.generic.GenericRecord;
 
 import java.io.File;
 import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * Utilities for generating Avro test data.
@@ -79,6 +82,98 @@ public class AvroTestUtil {
     return file.getAbsolutePath();
   }
 
+  public static String generateUnionSchema_WithNullValues() throws Exception {
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_long").type().longType().noDefault()
+            .name("d_float").type().floatType().noDefault()
+            .name("e_double").type().doubleType().noDefault()
+            .name("f_bytes").type().bytesType().noDefault()
+            .name("g_null").type().nullType().noDefault()
+            .name("h_boolean").type().booleanType().noDefault()
+            .name("i_union").type().optional().doubleType()
+            .endRecord();
+
+    final File file = File.createTempFile("avro-primitive-test", ".avro");
+    file.deleteOnExit();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    try {
+      writer.create(schema, file);
+
+      ByteBuffer bb = ByteBuffer.allocate(1);
+      bb.put(0, (byte) 1);
+
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+        record.put("c_long", (long) i);
+        record.put("d_float", (float) i);
+        record.put("e_double", (double) i);
+        record.put("f_bytes", bb);
+        record.put("g_null", null);
+        record.put("h_boolean", (i % 2 == 0));
+        record.put("i_union", (i % 2 == 0 ? (double) i : null));
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
+  public static String generateUnionSchema_WithNonNullValues() throws Exception {
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_long").type().longType().noDefault()
+            .name("d_float").type().floatType().noDefault()
+            .name("e_double").type().doubleType().noDefault()
+            .name("f_bytes").type().bytesType().noDefault()
+            .name("g_null").type().nullType().noDefault()
+            .name("h_boolean").type().booleanType().noDefault()
+            .name("i_union").type().unionOf().doubleType().and().longType().endUnion().noDefault()
+            .endRecord();
+
+    final File file = File.createTempFile("avro-primitive-test", ".avro");
+    file.deleteOnExit();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    try {
+      writer.create(schema, file);
+
+      ByteBuffer bb = ByteBuffer.allocate(1);
+      bb.put(0, (byte) 1);
+
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+        record.put("c_long", (long) i);
+        record.put("d_float", (float) i);
+        record.put("e_double", (double) i);
+        record.put("f_bytes", bb);
+        record.put("g_null", null);
+        record.put("h_boolean", (i % 2 == 0));
+        record.put("i_union", (i % 2 == 0 ? (double) i : (long) i));
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
   public static String generateSimpleEnumSchema_NoNullValues() throws Exception {
 
     final String[] symbols = { "E_SYM_A", "E_SYM_B", "E_SYM_C", "E_SYM_D" };
@@ -210,6 +305,185 @@ public class AvroTestUtil {
     return file.getAbsolutePath();
   }
 
+  public static String generateUnionNestedArraySchema_withNullValues() throws Exception {
+
+    final File file = File.createTempFile("avro-nested-test", ".avro");
+    file.deleteOnExit();
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_array").type().optional().array().items().record("my_record_1")
+            .namespace("foo.blah.org").fields()
+            .name("nested_1_string").type().optional().stringType()
+            .name("nested_1_int").type().optional().intType()
+            .endRecord()
+            .endRecord();
+
+    final Schema nestedSchema = schema.getField("c_array").schema();
+    final Schema arraySchema = nestedSchema.getTypes().get(1);
+    final Schema itemSchema = arraySchema.getElementType();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    writer.create(schema, file);
+
+    try {
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+
+        if (i % 2 == 0) {
+          GenericArray array = new GenericData.Array<String>(1, arraySchema);
+          final GenericRecord nestedRecord = new GenericData.Record(itemSchema);
+          nestedRecord.put("nested_1_string", "nested_1_string_" +  i);
+          nestedRecord.put("nested_1_int", i * i);
+          array.add(nestedRecord);
+          record.put("c_array", array);
+        }
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
+  public static String generateMapSchema_withNullValues() throws Exception {
+
+    final File file = File.createTempFile("avro-nested-test", ".avro");
+    file.deleteOnExit();
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_map").type().optional().map().values(Schema.create(Type.STRING))
+            .endRecord();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    writer.create(schema, file);
+
+    try {
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+
+        if (i % 2 == 0) {
+          Map<String, String> strMap = new HashMap<>();
+          strMap.put("key1", "nested_1_string_" +  i);
+          strMap.put("key2", "nested_1_string_" +  (i + 1 ));
+          record.put("c_map", strMap);
+        }
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
+  public static String generateMapSchemaComplex_withNullValues() throws Exception {
+
+    final File file = File.createTempFile("avro-nested-test", ".avro");
+    file.deleteOnExit();
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_map").type().optional().map().values(Schema.create(Type.STRING))
+            .name("d_map").type().optional().map().values(Schema.createArray(Schema.create(Type.DOUBLE)))
+            .endRecord();
+
+    final Schema arrayMapSchema = schema.getField("d_map").schema();
+    final Schema arrayItemSchema = arrayMapSchema.getTypes().get(1).getValueType();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    writer.create(schema, file);
+
+    try {
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+
+        if (i % 2 == 0) {
+          Map<String, String> c_map = new HashMap<>();
+          c_map.put("key1", "nested_1_string_" +  i);
+          c_map.put("key2", "nested_1_string_" +  (i + 1 ));
+          record.put("c_map", c_map);
+        } else {
+          Map<String, GenericArray> d_map = new HashMap<>();
+          GenericArray array = new GenericData.Array<String>(RECORD_COUNT, arrayItemSchema);
+          for (int j = 0; j < RECORD_COUNT; j++) {
+            array.add((double)j);
+          }
+          d_map.put("key1", array);
+          d_map.put("key2", array);
+
+          record.put("d_map", d_map);
+        }
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
+  public static String generateUnionNestedSchema_withNullValues() throws Exception {
+
+    final File file = File.createTempFile("avro-nested-test", ".avro");
+    file.deleteOnExit();
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringType().noDefault()
+            .name("b_int").type().intType().noDefault()
+            .name("c_record").type().optional().record("my_record_1")
+            .namespace("foo.blah.org").fields()
+            .name("nested_1_string").type().optional().stringType()
+            .name("nested_1_int").type().optional().intType()
+            .endRecord()
+            .endRecord();
+
+    final Schema nestedSchema = schema.getField("c_record").schema();
+    final Schema optionalSchema = nestedSchema.getTypes().get(1);
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    writer.create(schema, file);
+
+    try {
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_int", i);
+
+        if (i % 2 == 0) {
+          final GenericRecord nestedRecord = new GenericData.Record(optionalSchema);
+          nestedRecord.put("nested_1_string", "nested_1_string_" +  i);
+          nestedRecord.put("nested_1_int", i * i);
+          record.put("c_record", nestedRecord);
+        }
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
+
   public static String generateDoubleNestedSchema_NoNullValues() throws Exception {
 
     final File file = File.createTempFile("avro-double-nested-test", ".avro");
@@ -267,4 +541,36 @@ public class AvroTestUtil {
 
     return file.getAbsolutePath();
   }
+
+  public static String generateStringAndUtf8Data() throws Exception {
+
+    final Schema schema = SchemaBuilder.record("AvroRecordReaderTest")
+            .namespace("org.apache.drill.exec.store.avro")
+            .fields()
+            .name("a_string").type().stringBuilder().prop("avro.java.string", "String").endString().noDefault()
+            .name("b_utf8").type().stringType().noDefault()
+            .endRecord();
+
+    final File file = File.createTempFile("avro-primitive-test", ".avro");
+    file.deleteOnExit();
+
+    final DataFileWriter writer = new DataFileWriter(new GenericDatumWriter(schema));
+    try {
+      writer.create(schema, file);
+
+      ByteBuffer bb = ByteBuffer.allocate(1);
+      bb.put(0, (byte) 1);
+
+      for (int i = 0; i < RECORD_COUNT; i++) {
+        final GenericRecord record = new GenericData.Record(schema);
+        record.put("a_string", "a_" + i);
+        record.put("b_utf8", "b_" + i);
+        writer.append(record);
+      }
+    } finally {
+      writer.close();
+    }
+
+    return file.getAbsolutePath();
+  }
 }


[04/15] drill git commit: DRILL-3180: JDBC Storage Plugin updates.

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
deleted file mode 100644
index f3bf81d..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
+++ /dev/null
@@ -1,300 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.schema.Schema.TableType;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.linq4j.Extensions;
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.exec.store.AbstractSchema;
-
-import com.google.common.collect.ImmutableList;
-
-import org.apache.drill.exec.planner.logical.DrillTable;
-import org.apache.drill.exec.planner.logical.DynamicDrillTable;
-
-class MPJdbcClient {
-    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
-       .getLogger(MPJdbcClient.class);
-
-    private MPJdbcClientOptions clientOptions;
-    private Connection conn;
-    private DatabaseMetaData metadata;
-    private String uri;
-    private OdbcSchema defaultSchema;
-    private MPJdbcFormatPlugin plugin;
-    private String plugName;
-
-    public MPJdbcClient(String uri, MPJdbcClientOptions clientOptions,
-            MPJdbcFormatPlugin plugin) {
-        try {
-            Class.forName(clientOptions.getDriver()).newInstance();
-            this.clientOptions = clientOptions;
-
-            String user = this.clientOptions.getUser();
-            String passwd = this.clientOptions.getPassword();
-            this.plugin = plugin;
-            this.uri = uri;
-
-            if (user == null || user.length() == 0 || passwd.length() == 0) {
-                logger.info("username, password assumed to be in the uri");
-                this.conn = DriverManager.getConnection(uri);
-            } else {
-                this.conn = DriverManager.getConnection(uri, user, passwd);
-            }
-            this.metadata = this.conn.getMetaData();
-            this.plugName = plugin.getName();
-        } catch (InstantiationException e) {
-            // TODO Auto-generated catch block
-           new DrillRuntimeException(e);
-        } catch (IllegalAccessException e) {
-            // TODO Auto-generated catch block
-            new DrillRuntimeException(e);
-        } catch (ClassNotFoundException e) {
-            // TODO Auto-generated catch block
-            new DrillRuntimeException(e);
-        } catch (SQLException e) {
-            new DrillRuntimeException(e);
-        }
-    }
-
-    public Connection getConnection() {
-        return this.conn;
-    }
-
-    public Map<String, Integer> getSchemas() {
-        Map<String, Integer> lst = new HashMap<String, Integer>();
-        try {
-            ResultSet rs = this.metadata.getCatalogs();
-            while (rs.next()) {
-                Integer val = lst.get(rs.getString(1));
-                if (val == null) {
-                    lst.put(rs.getString(1), new Integer(1));
-                }
-            }
-
-        } catch (SQLException e) {
-            new DrillRuntimeException(e);
-        }
-        return lst;
-    }
-
-    public Set<String> getTables(String catalog) {
-        Set<String> lst = new HashSet<String>();
-
-        String[] typeList = { "TABLE", "VIEW" };
-        try {
-            ResultSet rs = this.metadata
-                    .getTables(catalog,null, null, null);
-            while (rs.next()) {
-                if (rs.getString(1) != null) {
-                  //lst.add(rs.getString(1) + "." + rs.getString("TABLE_NAME"));
-                  lst.add(rs.getString("TABLE_NAME"));
-                } else {
-                    lst.add(rs.getString("TABLE_NAME"));
-                }
-            }
-
-        } catch (SQLException e) {
-            throw new DrillRuntimeException(e);
-        }
-        return lst;
-    }
-
-    public List<String> getDatabases() {
-        List<String> lst = new ArrayList<String>();
-        try {
-            ResultSet rs = this.metadata.getCatalogs();
-            while (rs.next()) {
-                lst.add(rs.getString(0));
-            }
-        } catch (SQLException e) {
-            e.printStackTrace();
-        }
-        return lst;
-    }
-
-    public void close() {
-        // TODO Auto-generated method stub
-        try {
-            this.conn.close();
-        } catch (SQLException e) {
-            e.printStackTrace();
-        }
-
-    }
-
-    public OdbcSchema getSchema() {
-        List<String> l = new ArrayList<String>();
-        String currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                plugin).getCurrentSchema();
-        defaultSchema = new OdbcSchema(currentSchema);
-        return defaultSchema;
-    }
-
-    public OdbcSchema getSchema(String name) {
-        List<String> l = new ArrayList<String>();
-        OdbcSchema schema = new OdbcSchema(name);
-        return schema;
-    }
-
-    public class OdbcSchema extends AbstractSchema {
-
-        private Map<String, Integer> sub_schemas;
-        private String currentSchema;
-        private Set<String> tables;
-
-        public OdbcSchema(String name) {
-            super(ImmutableList.<String> of(), name);
-            /*currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                    plugin).getCurrentSchema();
-            if (currentSchema == null) {
-                currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                        plugin).getCurrentSchema();
-            }
-            */
-            if(name.equals("")) {
-              sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
-                  .getSchemas();
-            }
-            tables = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
-                    .getTables(name);
-        }
-
-        public OdbcSchema(List<String> parentSchemaPath, String name) {
-            super(parentSchemaPath, name);
-            currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                    plugin).getCurrentSchema();
-            if (currentSchema == null) {
-                currentSchema = "ROOT";
-            }
-            sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
-                    .getSchemas();
-            // TODO Auto-generated constructor stub
-        }
-
-        @Override
-        public String getTypeName() {
-            // TODO Auto-generated method stub
-            return "odbc";
-        }
-
-        @Override
-        public AbstractSchema getSubSchema(String name) {
-            if (sub_schemas == null) {
-                sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                        plugin).getSchemas();
-            }
-            Integer a = sub_schemas.get(name);
-            if (a == 1) {
-                return new OdbcSchema(name);
-            }
-            return null;
-        }
-
-        @Override
-        public Table getTable(String name) {
-            // TODO Auto-generated method stub
-          String tableName = null;
-          if(name.contains(".")) {
-            String[] val = name.split("\\.");
-            OdbcSchema sub = (OdbcSchema) this.getSubSchema(val[0]);
-            return sub.getTable(val[1]);
-          }
-          Iterator<String> iter = tables.iterator();
-          while(iter.hasNext()) {
-            tableName = iter.next();
-            if(tableName.equalsIgnoreCase(name)) {
-              break;
-            }
-            else {
-              tableName = null;
-            }
-          }
-          if(tableName == null) {
-            return null;
-          }
-          MPJdbcScanSpec spec = new MPJdbcScanSpec(this.name, tableName,"");
-          return new DynamicDrillTable(plugin, "odbc", spec);
-        }
-
-        @Override
-        public Set<String> getTableNames() {
-            // TODO Auto-generated method stub
-            Set<String> Tables = MPJdbcCnxnManager.getClient(uri, clientOptions,
-                    plugin).getTables(name);
-            return Tables;
-        }
-
-        @Override
-        public Set<String> getSubSchemaNames() {
-            // TODO Auto-generated method stub
-            sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
-                    .getSchemas();
-            return sub_schemas.keySet();
-        }
-
-        @Override
-        public Collection<Function> getFunctions(String name) {
-            // TODO Auto-generated method stub
-            return super.getFunctions(name);
-        }
-
-        @Override
-        public AbstractSchema getDefaultSchema() {
-            return MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
-                    .getDefaultSchema();
-        }
-
-    }
-
-    public String getCurrentSchema() {
-        // TODO Auto-generated method stub
-        try {
-            return this.conn.getCatalog();
-        } catch (SQLException e) {
-            // TODO Auto-generated catch block
-            e.printStackTrace();
-        }
-        return null;
-    }
-
-    public AbstractSchema getDefaultSchema() {
-        // TODO Auto-generated method stub
-        return defaultSchema;
-    }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
deleted file mode 100644
index 84b6348..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-public class MPJdbcClientOptions {
-  private String driver;
-  private String user;
-  private String passwd;
-
-  public MPJdbcClientOptions(String driver, String user, String passwd) {
-    this.driver = driver;
-    this.user = user;
-    this.passwd = passwd;
-  }
-
-  public MPJdbcClientOptions(MPJdbcFormatConfig storageConfig) {
-    this.driver = storageConfig.getDriver();
-    this.user = storageConfig.getUser();
-    this.passwd = storageConfig.getPasswd();
-  }
-
-  public String getDriver() {
-    // TODO Auto-generated method stub
-    return this.driver;
-  }
-
-  public String getUser() {
-    // TODO Auto-generated method stub
-    return this.user;
-  }
-
-  public String getPassword() {
-    // TODO Auto-generated method stub
-    return this.passwd;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
deleted file mode 100644
index 7e43c32..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.concurrent.TimeUnit;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-
-import org.apache.drill.exec.store.mpjdbc.MPJdbcClientOptions;
-
-public class MPJdbcCnxnManager {
-
-  private static final Logger logger = LoggerFactory
-      .getLogger(MPJdbcCnxnManager.class);
-  private static Cache<String, MPJdbcClient> uriClientMap;
-
-  static {
-    uriClientMap = CacheBuilder.newBuilder().maximumSize(5)
-        .expireAfterAccess(10, TimeUnit.MINUTES)
-        .removalListener(new uriCloser()).build();
-  }
-
-  private static class uriCloser implements RemovalListener<String, MPJdbcClient> {
-
-    @Override
-    public synchronized void onRemoval(
-        RemovalNotification<String, MPJdbcClient> removal) {
-      removal.getValue().close();
-      logger.debug("Closed connection to {}.", removal.getKey().toString());
-    }
-
-  }
-
-  public synchronized static MPJdbcClient getClient(String uri,
-      MPJdbcClientOptions clientOptions, MPJdbcFormatPlugin plugin) {
-    MPJdbcClient client = uriClientMap.getIfPresent(uri);
-    if (client == null) {
-      client = new MPJdbcClient(uri, clientOptions,plugin);
-      if (client.getConnection() != null) {
-        uriClientMap.put(uri, client);
-      } else {
-        return null;
-      }
-    }
-
-    return client;
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java
deleted file mode 100644
index 488a55d..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java
+++ /dev/null
@@ -1,235 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.List;
-
-import org.apache.drill.common.expression.BooleanOperator;
-import org.apache.drill.common.expression.FunctionCall;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
-import org.apache.drill.exec.store.mpjdbc.MPJdbcScanSpec;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.ImmutableList;
-
-public class MPJdbcFilterBuilder extends
-    AbstractExprVisitor<MPJdbcScanSpec, Void, RuntimeException> {
-  static final Logger logger = LoggerFactory
-      .getLogger(MPJdbcFilterBuilder.class);
-  final MPJdbcGroupScan groupScan;
-  final LogicalExpression le;
-  private boolean allExpressionsConverted = true;
-
-  public MPJdbcFilterBuilder(MPJdbcGroupScan groupScan,
-      LogicalExpression conditionExp) {
-    this.groupScan = groupScan;
-    this.le = conditionExp;
-  }
-
-  public MPJdbcScanSpec parseTree() {
-    MPJdbcScanSpec parsedSpec = le.accept(this, null);
-    if (parsedSpec != null) {
-      parsedSpec = mergeScanSpecs("booleanAnd", this.groupScan.getScanSpec(),
-          parsedSpec);
-    }
-    return parsedSpec;
-  }
-
-  private MPJdbcScanSpec mergeScanSpecs(String functionName,
-     MPJdbcScanSpec leftScanSpec, MPJdbcScanSpec rightScanSpec) {
-    List<String> newFilter;
-    switch (functionName) {
-    case "booleanAnd":
-      if (leftScanSpec.getFilters() != null
-          && rightScanSpec.getFilters() != null) {
-        /* newFilter = MongoUtils.andFilterAtIndex(leftScanSpec.getFilters(),
-            rightScanSpec.getFilters()); */
-      } else if (leftScanSpec.getFilters() != null) {
-        newFilter = leftScanSpec.getFilters();
-      } else {
-        newFilter = rightScanSpec.getFilters();
-      }
-      break;
-    case "booleanOr":
-     /* newFilter = OdbcUtils.orFilterAtIndex(leftScanSpec.getFilters(),
-          rightScanSpec.getFilters()); */
-    }
-    MPJdbcScanSpec mp =  new MPJdbcScanSpec(groupScan.getScanSpec().getDatabase(), groupScan
-        .getScanSpec().getTable(), groupScan.getScanSpec().getColumns());
-    return mp;
-  }
-
-  public boolean isAllExpressionsConverted() {
-    return allExpressionsConverted;
-  }
-
-  @Override
-  public MPJdbcScanSpec visitUnknown(LogicalExpression e, Void value)
-      throws RuntimeException {
-    allExpressionsConverted = false;
-    return null;
-  }
-
-  @Override
-  public MPJdbcScanSpec visitBooleanOperator(BooleanOperator op, Void value) {
-    List<LogicalExpression> args = op.args;
-    MPJdbcScanSpec nodeScanSpec = null;
-    String functionName = op.getName();
-    for (int i = 0; i < args.size(); ++i) {
-      switch (functionName) {
-      case "booleanAnd":
-      case "booleanOr":
-        if (nodeScanSpec == null) {
-          nodeScanSpec = args.get(i).accept(this, null);
-        } else {
-          MPJdbcScanSpec scanSpec = args.get(i).accept(this, null);
-          if (scanSpec != null) {
-            nodeScanSpec = mergeScanSpecs(functionName, nodeScanSpec, scanSpec);
-          } else {
-            allExpressionsConverted = false;
-          }
-        }
-        break;
-      }
-    }
-    return nodeScanSpec;
-  }
-
-  @Override
-  public MPJdbcScanSpec visitFunctionCall(FunctionCall call, Void value)
-      throws RuntimeException {
-    MPJdbcScanSpec nodeScanSpec = null;
-    String functionName = call.getName();
-    ImmutableList<LogicalExpression> args = call.args;
-    LogicalExpression nameVal = call.args.get(0);
-    LogicalExpression valueVal = null;
-    StringBuilder strBuilder = new StringBuilder();
-    if(call.args.size() >= 2) {
-      valueVal = call.args.get(1);
-    }
-    logger.info("Name Val:" + nameVal.toString());
-    logger.info("Value Val:" + valueVal.toString());
-
-    switch(functionName) {
-    case "equal":
-      break;
-     default:
-       break;
-    }
-    /*
-    if (OdbcCompareFunctionProcessor.isCompareFunction(functionName)) {
-      OdbcCompareFunctionProcessor processor = OdbcCompareFunctionProcessor
-          .process(call);
-      if (processor.isSuccess()) {
-        try {
-          nodeScanSpec = createOdbcScanSpec(processor.getFunctionName(),
-              processor.getPath(), processor.getValue());
-        } catch (Exception e) {
-          logger.error(" Failed to creare Filter ", e);
-          // throw new RuntimeException(e.getMessage(), e);
-        }
-      }
-    } else {
-      switch (functionName) {
-      case "booleanAnd":
-      case "booleanOr":
-        MPJdbcScanSpec leftScanSpec = args.get(0).accept(this, null);
-        MPJdbcScanSpec rightScanSpec = args.get(1).accept(this, null);
-        if (leftScanSpec != null && rightScanSpec != null) {
-          nodeScanSpec = mergeScanSpecs(functionName, leftScanSpec,
-              rightScanSpec);
-        } else {
-          allExpressionsConverted = false;
-          if ("booleanAnd".equals(functionName)) {
-            nodeScanSpec = leftScanSpec == null ? rightScanSpec : leftScanSpec;
-          }
-        }
-        break;
-      }
-    }
- */
-    if (nodeScanSpec == null) {
-      allExpressionsConverted = false;
-    }
-
-    return nodeScanSpec;
-  }
-
-  private MPJdbcScanSpec createOdbcScanSpec(String functionName,
-      SchemaPath field, Object fieldValue) throws ClassNotFoundException,
-      Exception {
-    // extract the field name
-    String fieldName = field.getAsUnescapedPath();
-    /*
-    OdbcCompareOp compareOp = null;
-    switch (functionName) {
-    case "equal":
-      compareOp = OdbcCompareOp.EQUAL;
-      break;
-    case "not_equal":
-      compareOp = OdbcCompareOp.NOT_EQUAL;
-      break;
-    case "greater_than_or_equal_to":
-      compareOp = OdbcCompareOp.GREATER_OR_EQUAL;
-      break;
-    case "greater_than":
-      compareOp = OdbcCompareOp.GREATER;
-      break;
-    case "less_than_or_equal_to":
-      compareOp = OdbcCompareOp.LESS_OR_EQUAL;
-      break;
-    case "less_than":
-      compareOp = OdbcCompareOp.LESS;
-      break;
-    case "isnull":
-    case "isNull":
-    case "is null":
-      compareOp = OdbcCompareOp.IFNULL;
-      break;
-    case "isnotnull":
-    case "isNotNull":
-    case "is not null":
-      compareOp = OdbcCompareOp.IFNOTNULL;
-      break;
-    }
-
-    if (compareOp != null) {
-      BasicDBObject queryFilter = new BasicDBObject();
-      if (compareOp == OdbcCompareOp.IFNULL) {
-        queryFilter.put(fieldName,
-            new BasicDBObject(OdbcCompareOp.EQUAL.getCompareOp(), null));
-      } else if (compareOp == OdbcCompareOp.IFNOTNULL) {
-        queryFilter.put(fieldName,
-            new BasicDBObject(OdbcCompareOp.NOT_EQUAL.getCompareOp(), null));
-      } else {
-        queryFilter.put(fieldName, new BasicDBObject(compareOp.getCompareOp(),
-            fieldValue));
-      }
-      return new MPJdbcScanSpec(groupScan.getScanSpec().getDbName(), groupScan
-          .getScanSpec().getCollectionName(), queryFilter);
-    }
-    */
-    return null;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
deleted file mode 100644
index c0564d0..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.plan.RelOptRuleOperand;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeField;
-import org.apache.calcite.rex.RexNode;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.exec.planner.logical.DrillOptiq;
-import org.apache.drill.exec.planner.logical.DrillParseContext;
-import org.apache.drill.exec.planner.logical.RelOptHelper;
-import org.apache.drill.exec.planner.physical.FilterPrel;
-import org.apache.drill.exec.planner.physical.ScanPrel;
-import org.apache.drill.exec.store.StoragePluginOptimizerRule;
-
-public class MPJdbcFilterRule extends StoragePluginOptimizerRule {
-  public static final StoragePluginOptimizerRule INSTANCE = new MPJdbcFilterRule();
-  public MPJdbcFilterRule(RelOptRuleOperand operand, String description) {
-    super(operand, description);
-    // TODO Auto-generated constructor stub
-  }
-
-  public MPJdbcFilterRule() {
-    // TODO Auto-generated constructor stub
-    super(RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)), "MPJdbcFilterRule");
-  }
-
-  @Override
-  public void onMatch(RelOptRuleCall call) {
-    // TODO Auto-generated method stub
-    final ScanPrel scan = (ScanPrel) call.rel(1);
-    final FilterPrel filter = (FilterPrel) call.rel(0);
-    final RexNode condition = filter.getCondition();
-    final LogicalExpression conditionExp = DrillOptiq.toDrill(new DrillParseContext(), scan, condition);
-    MPJdbcGroupScan grpScan= (MPJdbcGroupScan) scan.getGroupScan();
-    MPJdbcFilterBuilder builder = new MPJdbcFilterBuilder(grpScan,conditionExp);
-    MPJdbcScanSpec result = builder.parseTree();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
deleted file mode 100644
index 8edce3e..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import org.apache.drill.common.logical.StoragePluginConfig;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.databind.ObjectMapper;
-
-@JsonTypeName(MPJdbcFormatConfig.NAME)
-public class MPJdbcFormatConfig extends StoragePluginConfig {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
-      .getLogger(MPJdbcFormatConfig.class);
-  public static final String NAME = "jdbc";
-
-  @JsonIgnore
-  private String driver;
-  @JsonIgnore
-  private String uri;
-  @JsonIgnore
-  private String username;
-  @JsonIgnore
-  private String password;
-
-  @JsonCreator
-  public MPJdbcFormatConfig(@JsonProperty("driver") String driver,
-      @JsonProperty("uri") String uri,
-      @JsonProperty("username") String username,
-      @JsonProperty("password") String password) {
-    this.driver = driver == null ? "" : driver;
-    this.uri = uri == null ? "jdbc://" : uri;
-    this.username = username == null ? "" : username;
-    this.password = password == null ? "" : password;
-
-  }
-
-  @Override
-  public boolean equals(Object o) {
-    if (this == o) {
-      return true;
-    }
-    if (o == null || getClass() != o.getClass()) {
-      return false;
-    }
-
-    MPJdbcFormatConfig that = (MPJdbcFormatConfig) o;
-
-    if (uri != null ? !uri.equals(that.uri) : that.uri != null) {
-      return false;
-    }
-
-    return true;
-  }
-
-  @JsonProperty("driver")
-  public String getDriver() {
-    return this.driver;
-  }
-
-  @JsonProperty("uri")
-  public String getUri() {
-    return this.uri;
-  }
-
-  @JsonProperty("username")
-  public String getUser() {
-    return this.username;
-  }
-
-  @JsonProperty("password")
-  public String getPasswd() {
-    return this.password;
-  }
-
-  @Override
-  public int hashCode() {
-    ObjectMapper mapper = new ObjectMapper();
-    try {
-      String outval = mapper.writeValueAsString(this);
-      logger.info("FormatConfigHashCode:" + outval);
-
-      return outval.hashCode();
-    } catch (JsonProcessingException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-      return uri.hashCode();
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
deleted file mode 100644
index 5a0dd4b..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
+++ /dev/null
@@ -1,170 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.io.IOException;
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.ResultSet;
-import java.sql.ResultSetMetaData;
-import java.sql.SQLException;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.apache.calcite.schema.Table;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.schema.Schema;
-import org.apache.calcite.schema.SchemaPlus;
-import org.apache.calcite.schema.Statistic;
-import org.apache.calcite.schema.Schema.TableType;
-import org.apache.calcite.schema.Function;
-import org.apache.calcite.linq4j.Extensions;
-import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.logical.FormatPluginConfig;
-import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.AbstractWriter;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.rpc.user.UserSession;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.AbstractStoragePlugin;
-import org.apache.drill.exec.store.SchemaConfig;
-import org.apache.drill.exec.store.StoragePluginOptimizerRule;
-import org.apache.drill.exec.store.dfs.DrillFileSystem;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP;
-import org.apache.drill.exec.store.mock.MockStorageEngine;
-import org.apache.drill.exec.store.mock.MockStorageEngineConfig;
-import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
-import org.apache.drill.exec.store.mpjdbc.MPJdbcClient.OdbcSchema;
-import org.apache.calcite.plan.RelOptRuleCall;
-import org.apache.calcite.rel.type.RelDataType;
-import org.apache.calcite.rel.type.RelDataTypeFactory;
-
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.core.JsonProcessingException;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-
-public class MPJdbcFormatPlugin extends AbstractStoragePlugin {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
-      .getLogger(MPJdbcFormatPlugin.class);
-
-  private final MPJdbcFormatConfig storageConfig;
-  protected String name = "odbc";
-  private final DrillbitContext context;
-
-  public MPJdbcFormatPlugin(MPJdbcFormatConfig storageConfig,
-      DrillbitContext context, String name) {
-    this.context = context;
-    this.storageConfig = storageConfig;
-    ObjectMapper mapper = new ObjectMapper();
-    try {
-      String result = mapper.writeValueAsString(storageConfig);
-      logger.info(result);
-    } catch (JsonProcessingException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-    this.name = name;
-  }
-
-  @Override
-  public void registerSchemas(SchemaConfig config, SchemaPlus parent) {
-    if(storageConfig == null) {
-       logger.info("StorageConfig is null");
-    }
-    MPJdbcClientOptions options = new MPJdbcClientOptions(storageConfig);
-    MPJdbcClient client = MPJdbcCnxnManager.getClient(storageConfig.getUri(),
-        options,this);
-    Connection conn = (client == null) ? null : client.getConnection();
-    Map<String, Integer> schemas;
-    if(client == null) {
-      logger.info("Could not create client...");
-    }
-    OdbcSchema o = client.getSchema();
-    SchemaPlus tl = parent.add(this.name, o);
-    try {
-      schemas = client.getSchemas();
-      Set<Entry<String, Integer>> a = schemas.entrySet();
-      Iterator<Entry<String, Integer>> aiter = a.iterator();
-      while (aiter.hasNext()) {
-        Entry<String, Integer> val = aiter.next();
-        String catalog = val.getKey();
-        OdbcSchema sc = client.getSchema(catalog);
-        tl.add(catalog, sc);
-      }
-    } catch (Exception e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  @Override
-  public MPJdbcFormatConfig getConfig() {
-    logger.info("MPJdbcFormatPlugin:getConfig called");
-    logger.info(storageConfig.toString());
-    return storageConfig;
-  }
-
-  public DrillbitContext getContext() {
-    return this.context;
-  }
-
-  public String getName() {
-      return this.name;
-  }
-
-  @Override
-  public boolean supportsRead() {
-    return true;
-  }
-/*
-  @Override
-  public AbstractGroupScan getPhysicalScan(String userName,JSONOptions selection)
-      throws IOException {
-    MPJdbcScanSpec odbcScanSpec = selection.getListWith(new ObjectMapper(),
-        new TypeReference<MPJdbcScanSpec>() {
-        });
-    return new MPJdbcGroupScan(userName,this, odbcScanSpec, null);
-  }
-  */
-  @Override
-  public AbstractGroupScan getPhysicalScan(String userName,JSONOptions selection,List<SchemaPath> columns)
-      throws IOException {
-    MPJdbcScanSpec mPJdbcScanSpec = selection.getListWith(new ObjectMapper(),
-        new TypeReference<MPJdbcScanSpec>() {
-        });
-    return new MPJdbcGroupScan(userName,this, mPJdbcScanSpec, columns);
-  }
-
-  @Override
-  public Set<StoragePluginOptimizerRule> getOptimizerRules() {
-    // TODO Auto-generated method stub
-    return ImmutableSet.of(MPJdbcFilterRule.INSTANCE);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
deleted file mode 100644
index a16f8c8..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
+++ /dev/null
@@ -1,181 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.exec.physical.EndpointAffinity;
-import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
-import org.apache.drill.exec.physical.base.AbstractGroupScan;
-import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.ScanStats;
-import org.apache.drill.exec.physical.base.SubScan;
-import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
-import org.apache.drill.exec.store.schedule.CompleteFileWork;
-import org.apache.drill.common.expression.SchemaPath;
-
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.ListMultimap;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-public class MPJdbcGroupScan extends AbstractGroupScan {
-
-  private MPJdbcFormatPlugin plugin;
-  private MPJdbcFormatConfig pluginConfig;
-  private MPJdbcScanSpec mPJdbcScanSpec;
-  private List<SchemaPath> columns;
-  private String userName;
-  private Map<Integer, List<MPJdbcScanSpec>> endpointFragmentMapping;
-
-  public MPJdbcGroupScan(String userName,MPJdbcFormatPlugin storagePlugin, MPJdbcScanSpec scanSpec,
-      List<SchemaPath> columns) {
-    super(userName);
-    this.plugin = storagePlugin;
-    this.pluginConfig = storagePlugin.getConfig();
-    this.mPJdbcScanSpec = scanSpec;
-    this.userName = userName;
-    this.columns = columns == null || columns.size() == 0 ? ALL_COLUMNS
-        : columns;
-  }
-
-  public MPJdbcGroupScan(MPJdbcGroupScan that) {
-    super(that);
-    this.columns = that.columns;
-    this.plugin = that.plugin;
-    this.endpointFragmentMapping = that.endpointFragmentMapping;
-    this.pluginConfig = that.pluginConfig;
-    this.mPJdbcScanSpec = that.mPJdbcScanSpec;
-  }
-
-  @Override
-  public SubScan getSpecificScan(int minorFragmentId)
-      throws ExecutionSetupException {
-    // TODO Auto-generated method stub
-    return new MPJdbcSubScan(plugin,userName, pluginConfig,
-        endpointFragmentMapping.get(minorFragmentId), columns);
-  }
-
-  @Override
-  public int getMaxParallelizationWidth() {
-    // TODO Auto-generated method stub
-    return -1;
-  }
-
-  @Override
-  public String getDigest() {
-    // TODO Auto-generated method stub
-    return toString();
-  }
-
-  @Override
-  public GroupScan clone(List<SchemaPath> columns) {
-    MPJdbcGroupScan newScan = new MPJdbcGroupScan(userName,plugin, mPJdbcScanSpec, columns);
-    return newScan;
-
-  }
-
-  @Override
-  public ScanStats getScanStats() {
-    // TODO Auto-generated method stub
-    return ScanStats.TRIVIAL_TABLE;
-  }
-
-  @Override
-  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children)
-      throws ExecutionSetupException {
-    Preconditions.checkArgument(children.isEmpty());
-    return new MPJdbcGroupScan(this);
-    // TODO Auto-generated method stub
-  }
-  @Override
-  @JsonIgnore
-  public boolean canPushdownProjects(List<SchemaPath> columns) {
-    this.columns = columns;
-    return true;
-  }
-
-  @Override
-  public List<EndpointAffinity> getOperatorAffinity() {
-    Map<String, DrillbitEndpoint> endpointMap = new HashMap<String, DrillbitEndpoint>();
-    for (DrillbitEndpoint ep : plugin.getContext().getBits()) {
-      endpointMap.put(ep.getAddress(), ep);
-    }
-
-    Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<DrillbitEndpoint, EndpointAffinity>();
-    DrillbitEndpoint ep = endpointMap.get(plugin.getConfig().getUri());
-    if (ep != null) {
-      EndpointAffinity affinity = affinityMap.get(ep);
-      if (affinity == null) {
-        affinityMap.put(ep, new EndpointAffinity(ep, 1));
-      } else {
-        affinity.addAffinity(1);
-      }
-    }
-    return Lists.newArrayList(affinityMap.values());
-  }
-
-  @Override
-  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
-    final int numSlots = incomingEndpoints.size();
-    int totalAssignmentsTobeDone = 1;
-    Preconditions.checkArgument(numSlots <= totalAssignmentsTobeDone, String
-        .format("Incoming endpoints %d is greater than number of chunks %d",
-            numSlots, totalAssignmentsTobeDone));
-    final int minPerEndpointSlot = (int) Math
-        .floor((double) totalAssignmentsTobeDone / numSlots);
-    final int maxPerEndpointSlot = (int) Math
-        .ceil((double) totalAssignmentsTobeDone / numSlots);
-    /* Map for (index,endpoint)'s */
-    endpointFragmentMapping = Maps.newHashMapWithExpectedSize(numSlots);
-    /* Reverse mapping for above indexes */
-    Map<String, Queue<Integer>> endpointHostIndexListMap = Maps.newHashMap();
-    /*
-     * Initialize these two maps
-     */
-    for (int i = 0; i < numSlots; ++i) {
-      List<MPJdbcScanSpec> val = new ArrayList<MPJdbcScanSpec>(maxPerEndpointSlot);
-      val.add(this.mPJdbcScanSpec);
-      endpointFragmentMapping.put(i, val);
-      String hostname = incomingEndpoints.get(i).getAddress();
-      Queue<Integer> hostIndexQueue = endpointHostIndexListMap.get(hostname);
-      if (hostIndexQueue == null) {
-        hostIndexQueue = Lists.newLinkedList();
-        endpointHostIndexListMap.put(hostname, hostIndexQueue);
-      }
-      hostIndexQueue.add(i);
-    }
-  }
-
-  public MPJdbcScanSpec getScanSpec() {
-    return this.mPJdbcScanSpec;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
deleted file mode 100644
index 498b7fd..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
+++ /dev/null
@@ -1,471 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import org.apache.drill.common.exceptions.DrillRuntimeException;
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.common.types.TypeProtos;
-import org.apache.drill.common.types.TypeProtos.DataMode;
-import org.apache.drill.common.types.TypeProtos.MajorType;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.expr.TypeHelper;
-import org.apache.drill.exec.expr.holders.DateHolder;
-import org.apache.drill.exec.expr.holders.Decimal38DenseHolder;
-import org.apache.drill.exec.expr.holders.VarCharHolder;
-import org.apache.drill.exec.memory.OutOfMemoryException;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.ops.OperatorContext;
-import org.apache.drill.exec.physical.impl.OutputMutator;
-import org.apache.drill.exec.record.MaterializedField;
-import org.apache.drill.exec.record.MaterializedField.Key;
-import org.apache.drill.exec.store.AbstractRecordReader;
-import org.apache.drill.exec.vector.AllocationHelper;
-import org.apache.drill.exec.vector.BigIntVector;
-import org.apache.drill.exec.vector.DateVector;
-import org.apache.drill.exec.vector.Decimal38DenseVector;
-import org.apache.drill.exec.vector.FixedWidthVector;
-import org.apache.drill.exec.vector.Float8Vector;
-import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.NullableBigIntVector;
-import org.apache.drill.exec.vector.NullableDateVector;
-import org.apache.drill.exec.vector.NullableDecimal38DenseVector;
-import org.apache.drill.exec.vector.NullableFloat8Vector;
-import org.apache.drill.exec.vector.NullableIntVector;
-import org.apache.drill.exec.vector.NullableTimeStampVector;
-import org.apache.drill.exec.vector.NullableTimeVector;
-import org.apache.drill.exec.vector.NullableVar16CharVector.Mutator;
-import org.apache.drill.exec.vector.NullableVarCharVector;
-import org.apache.drill.exec.vector.TimeStampVector;
-import org.apache.drill.exec.vector.TimeVector;
-import org.apache.drill.exec.vector.ValueHolderHelper;
-import org.apache.drill.exec.vector.ValueVector;
-import org.apache.drill.exec.vector.VarBinaryVector;
-import org.apache.drill.exec.vector.VarCharVector;
-import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
-import org.joda.time.DateTime;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.Lists;
-
-import io.netty.buffer.ByteBuf;
-import io.netty.buffer.DrillBuf;
-
-import java.sql.Connection;
-import java.sql.DatabaseMetaData;
-import java.sql.DriverManager;
-import java.sql.ResultSetMetaData;
-import java.sql.Statement;
-import java.sql.ResultSet;
-import java.sql.SQLException;
-import java.sql.Types;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-public class MPJdbcRecordReader extends AbstractRecordReader {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
-      .getLogger(MPJdbcRecordReader.class);
-
-  private ResultSet rec;
-  private VectorContainerWriter writer;
-  private FragmentContext fc;
-  private MPJdbcSubScan scanSpec;
-  private MPJdbcFormatPlugin plugin;
-  private List<MPJdbcScanSpec> scanList;
-  private MPJdbcFormatConfig config;
-  private Connection conn;
-  private Statement statement;
-  private String table;
-  private String database;
-  protected List<ValueVector> vectors = Lists.newArrayList();
-  private int col_cnt = 0;
-  private MajorType.Builder t;
-  private OutputMutator outputMutator;
-  private ResultSetMetaData meta;
-  private OperatorContext operatorContext;
-  private String columns;
-  private List<String> filters;
-
-  public MPJdbcRecordReader(FragmentContext fragmentContext, MPJdbcSubScan scan) {
-    fc = fragmentContext;
-    scanSpec = scan;
-    // TODO Auto-generated constructor stub
-    this.plugin = scanSpec.getPlugin();
-    this.scanList = scanSpec.getScanList();
-    this.config = scanSpec.getConfig();
-    MPJdbcClientOptions options = new MPJdbcClientOptions(config);
-    MPJdbcClient client = MPJdbcCnxnManager.getClient(config.getUri(), options,
-        this.plugin);
-    conn = client.getConnection();
-    Iterator<MPJdbcScanSpec> iter = scanList.iterator();
-    while (iter.hasNext()) {
-      MPJdbcScanSpec o = iter.next();
-      table = o.getTable();
-      database = o.getDatabase();
-      List<SchemaPath> ColList = scan.getColumns();
-      Iterator<SchemaPath> collist_iter = ColList.iterator();
-      StringBuilder b = new StringBuilder();
-      while(collist_iter.hasNext()) {
-          SchemaPath val = collist_iter.next();
-          b.append(val.getAsUnescapedPath().trim());
-          if(collist_iter.hasNext()) {
-              b.append(",");
-          }
-      }
-      columns = b.toString();
-      filters = o.getFilters();
-    }
-    try {
-      statement = conn.createStatement();
-      rec = statement.executeQuery("SELECT " + this.columns + " FROM " + database.trim() + "." + table.trim());
-    } catch (SQLException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-  }
-
-  @Override
-  public void setup(OutputMutator output) throws ExecutionSetupException {
-    try {
-      meta = rec.getMetaData();
-      col_cnt = meta.getColumnCount();
-      Class cls = null;
-      for (int i = 1; i <= col_cnt; i++) {
-        String column_label = meta.getColumnLabel(i);
-        int types = meta.getColumnType(i);
-        int isnullable = meta.isNullable(i);
-        int width = meta.getPrecision(i);
-        int scale = meta.getScale(i);
-        MaterializedField field = null;
-        switch (types) {
-        case java.sql.Types.BIGINT:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.BIGINT);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableBigIntVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, BigIntVector.class));
-          }
-          break;
-        case Types.DATE:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.DATE);
-          cls = org.apache.drill.exec.vector.DateVector.class;
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableDateVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, DateVector.class));
-          }
-          break;
-        case Types.DECIMAL:
-          t = MajorType.newBuilder().setMinorType(
-              TypeProtos.MinorType.DECIMAL38DENSE);
-          t.setMode(DataMode.OPTIONAL);
-          field = MaterializedField.create(column_label, t.build());
-          vectors.add(output.addField(field, Decimal38DenseVector.class));
-          break;
-        case Types.DOUBLE:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.BIGINT);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableBigIntVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, BigIntVector.class));
-          }
-          break;
-        case Types.FLOAT:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FLOAT8);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableFloat8Vector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, Float8Vector.class));
-          }
-          break;
-        case Types.INTEGER:
-        case Types.SMALLINT:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.INT);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableIntVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, IntVector.class));
-          }
-          break;
-        case Types.LONGNVARCHAR:
-        case Types.LONGVARCHAR:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
-          t.setWidth(width);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableVarCharVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, VarCharVector.class));
-          }
-          break;
-        case Types.LONGVARBINARY:
-          break;
-        case Types.CHAR:
-        case Types.NCHAR:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
-          t.setWidth(width);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableVarCharVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, VarCharVector.class));
-          }
-          break;
-        case Types.NUMERIC:
-          t = MajorType.newBuilder().setMinorType(
-              TypeProtos.MinorType.DECIMAL38DENSE);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field,
-                NullableDecimal38DenseVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, Decimal38DenseVector.class));
-          }
-          break;
-        case Types.NVARCHAR:
-        case Types.VARCHAR:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
-          t.setWidth(width);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableVarCharVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, VarCharVector.class));
-          }
-          break;
-        case Types.TIME:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.TIME);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableTimeVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, TimeVector.class));
-          }
-          break;
-        case Types.TIMESTAMP:
-          t = MajorType.newBuilder().setMinorType(
-              TypeProtos.MinorType.TIMESTAMP);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableTimeStampVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, TimeStampVector.class));
-          }
-          break;
-        default:
-          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
-          t.setWidth(width);
-          if (isnullable == 1) {
-            t.setMode(DataMode.OPTIONAL);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, NullableVarCharVector.class));
-          } else {
-            t.setMode(DataMode.REQUIRED);
-            field = MaterializedField.create(column_label, t.build());
-            vectors.add(output.addField(field, VarCharVector.class));
-          }
-          break;
-        }
-      }
-      this.outputMutator = output;
-
-    } catch (SQLException | SchemaChangeException e) {
-      // TODO Auto-generated catch block
-      throw new ExecutionSetupException(e);
-    }
-  }
-
-  @Override
-  public void setOperatorContext(OperatorContext operatorContext) {
-    this.operatorContext = operatorContext;
-  }
-
-  @Override
-  public int next() {
-    // TODO Auto-generated method stub
-    int counter = 0;
-    int pos = 1;
-    int prec = 0;
-    Boolean b = true;
-    try {
-      while (counter < 65536 && b == true) {
-        b = rec.next();
-        if(b == false) {
-            break;
-        }
-        for (ValueVector vv : vectors) {
-          String val = rec.getString(pos);
-          byte[] record = val.getBytes(Charsets.UTF_8);
-          if (vv.getClass().equals(NullableVarCharVector.class)) {
-            NullableVarCharVector v = (NullableVarCharVector) vv;
-            v.getMutator().setSafe(counter, record, 0, record.length);
-            v.getMutator().setValueLengthSafe(counter, record.length);
-          } else if (vv.getClass().equals(VarCharVector.class)) {
-            VarCharVector v = (VarCharVector) vv;
-            v.getMutator().setSafe(counter, record, 0, record.length);
-            v.getMutator().setValueLengthSafe(counter, record.length);
-          } else if (vv.getClass().equals(BigIntVector.class)) {
-            BigIntVector v = (BigIntVector) vv;
-            v.getMutator().setSafe(counter, rec.getLong(pos));
-          } else if (vv.getClass().equals(NullableBigIntVector.class)) {
-            NullableBigIntVector v = (NullableBigIntVector) vv;
-            v.getMutator().setSafe(counter, rec.getLong(pos));
-          } else if (vv.getClass().equals(IntVector.class)) {
-            IntVector v = (IntVector) vv;
-            v.getMutator().setSafe(counter, rec.getInt(pos));
-          } else if (vv.getClass().equals(NullableIntVector.class)) {
-            NullableIntVector v = (NullableIntVector) vv;
-            v.getMutator().setSafe(counter, rec.getInt(pos));
-          } else if (vv.getClass().equals(DateVector.class)) {
-            DateVector v = (DateVector) vv;
-            long dtime = DateTime.parse(val).toDate().getTime(); // DateTime.parse(val).toDateTime().getMillis();
-            v.getMutator().setSafe(counter, dtime);
-          } else if (vv.getClass().equals(NullableDateVector.class)) {
-            NullableDateVector v = (NullableDateVector) vv;
-            if (rec.wasNull()) {
-              v.getMutator().setNull(counter);
-            } else {
-              long dtime = DateTime.parse(val).toDate().getTime();
-              v.getMutator().setSafe(counter, dtime);
-            }
-          } else if (vv.getClass().equals(Decimal38DenseVector.class)) {
-            Decimal38DenseVector v = (Decimal38DenseVector) vv;
-            java.math.BigDecimal d = rec.getBigDecimal(pos);
-          } else if (vv.getClass().equals(NullableDecimal38DenseVector.class)) {
-            NullableDecimal38DenseVector v = (NullableDecimal38DenseVector) vv;
-            java.math.BigDecimal d = rec.getBigDecimal(pos);
-          } else {
-            NullableVarCharVector v = (NullableVarCharVector) vv;
-            v.getMutator().setSafe(counter, record, 0, record.length);
-            v.getMutator().setValueLengthSafe(counter, record.length);
-          }
-          pos++;
-        }
-        pos = 1;
-        counter++;
-      }
-    } catch (SQLException e) {
-      // TODO Auto-generated catch block
-      throw new DrillRuntimeException(e);
-    }
-    //logger.info("Number of rows returned from JDBC: " + counter);
-    for (ValueVector vv : vectors) {
-      vv.getMutator().setValueCount(counter > 0 ? counter : 0);
-    }
-    return counter>0 ? counter : 0;
-  }
-
-  @Override
-  public void allocate(Map<Key, ValueVector> vectorMap)
-      throws OutOfMemoryException {
-    int prec = 0;
-    try {
-      for (ValueVector vv : vectorMap.values()) {
-        if (vv.getClass().equals(NullableVarCharVector.class)) {
-                NullableVarCharVector v = (NullableVarCharVector) vv;
-                prec = v.getField().getWidth();
-                if(prec > 0) {
-                   AllocationHelper.allocate(v, 65536, prec);
-                } else {
-                   AllocationHelper.allocate(v, 65536, 2000);
-                }
-              } else if (vv.getClass().equals(VarCharVector.class)) {
-                VarCharVector v = (VarCharVector) vv;
-                prec = v.getField().getWidth();
-                if(prec > 0) {
-                    AllocationHelper.allocate(v, 65536, prec);
-                 } else {
-                    AllocationHelper.allocate(v, 65536, 2000);
-                 }
-              } else if (vv.getClass().equals(BigIntVector.class)) {
-                BigIntVector v = (BigIntVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(NullableBigIntVector.class)) {
-                NullableBigIntVector v = (NullableBigIntVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(DateVector.class)) {
-                DateVector v = (DateVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(NullableDateVector.class)) {
-                NullableDateVector v = (NullableDateVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(Decimal38DenseVector.class)) {
-                Decimal38DenseVector v = (Decimal38DenseVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(NullableDecimal38DenseVector.class)) {
-                NullableDecimal38DenseVector v = (NullableDecimal38DenseVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(IntVector.class)) {
-                IntVector v = (IntVector) vv;
-                v.allocateNew(65536);
-              } else if (vv.getClass().equals(NullableIntVector.class)) {
-                NullableIntVector v = (NullableIntVector) vv;
-                v.allocateNew(65536);
-              }
-      }
-    } catch (NullPointerException e) {
-      throw new OutOfMemoryException();
-    }
-  }
-
-  @Override
-  public void cleanup() {
-    // TODO Auto-generated method stub
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
deleted file mode 100644
index fae0e81..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
+++ /dev/null
@@ -1,76 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.List;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class MPJdbcScanSpec {
-
-  private String database;
-  private String table;
-  private String columns;
-
-  @JsonIgnore
-  private List<String> filters;
-
-  @JsonCreator
-  public MPJdbcScanSpec(@JsonProperty("database") String database,
-      @JsonProperty("table") String table, @JsonProperty("columns") String columns) {
-    this.database = database;
-    this.table = table;
-    this.columns = columns;
-  }
-
-  public MPJdbcScanSpec(String database, String table, List<String> filters, String columns) {
-    this.database = database;
-    this.table = table;
-    this.filters = filters;
-    this.columns = columns;
-  }
-
-  public String getDatabase() {
-    return this.database;
-  }
-
-  public String getTable() {
-    return this.table;
-  }
-
-  public List<String> getFilters() {
-    return this.filters;
-  }
-
-  public String getColumns() {
-    return this.columns;
-  }
-  @Override
-  public String toString() {
-    return "MPJdbcScanSpec [Database=" + database + ", table=" + table
-        + ", columns=" + columns + ", filters=" + filters + "]";
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    // TODO Auto-generated method stub
-    return super.equals(obj);
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
deleted file mode 100644
index f3169fc..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.drill.exec.store.mpjdbc;
-
-import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-/**
- * Stores the workspace related config. A workspace has: - location which is a
- * path. - writable flag to indicate whether the location supports creating new
- * tables. - default storage format for new tables created in this workspace.
- */
-
-public class MPJdbcSchemaConfig {
-
-  /** Default workspace is a root directory which supports read, but not write. */
-  public static final MPJdbcSchemaConfig DEFAULT = new MPJdbcSchemaConfig("jdbc://", "",
-      "");
-
-  private final String uri;
-  private final String username;
-  private final String passwd;
-
-  public MPJdbcSchemaConfig(@JsonProperty("uri") String uri,
-      @JsonProperty("username") String username,
-      @JsonProperty("passwd") String passwd) {
-    this.uri = uri;
-    this.username = username;
-    this.passwd = passwd;
-  }
-
-  public String getUri() {
-    return uri;
-  }
-
-  public boolean isWritable() {
-    return false;
-  }
-
-  public String getUsername() {
-    return username;
-  }
-
-  public String getPassword() {
-    return passwd;
-  }
-
-  @Override
-  public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    }
-
-    if (obj == null || !(obj instanceof MPJdbcSchemaConfig)) {
-      return false;
-    }
-
-    MPJdbcSchemaConfig that = (MPJdbcSchemaConfig) obj;
-    return ((this.uri == null && that.uri == null) || this.uri.equals(that.uri))
-        && this.passwd == that.passwd
-        && ((this.username == null && that.username == null) || this.username
-            .equals(that.username));
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
deleted file mode 100644
index 0e28c8d..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
+++ /dev/null
@@ -1,23 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-public class MPJdbcSchemaFilter {
-public MPJdbcSchemaFilter() {
-// TODO Auto-generated constructor stub
-}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
deleted file mode 100644
index 8c92533..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
+++ /dev/null
@@ -1,55 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-import org.apache.drill.exec.physical.base.AbstractSubScan;
-import org.apache.drill.exec.store.mpjdbc.MPJdbcSchemaFilter;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-public class MPJdbcSchemaSubScan extends AbstractSubScan {
-private final String table;
-private final MPJdbcSchemaFilter filter;
-private final String userName;
-
-  @JsonCreator
-  public MPJdbcSchemaSubScan(@JsonProperty("userName") String userName,
-       @JsonProperty("table") String table,
-       @JsonProperty("filter") MPJdbcSchemaFilter filter) {
-    super(userName);
-    this.table = table;
-    this.filter = filter;
-    this.userName = userName;
-  }
-  @JsonProperty("table")
-  public String getTable() {
-    return table;
-  }
-  @JsonProperty("filter")
-  public MPJdbcSchemaFilter getFilter() {
-    return filter;
-  }
-  @JsonProperty("userName")
-  public String getUserName() {
-    return this.userName;
-  }
-@Override
-public int getOperatorType() {
-// TODO Auto-generated method stub
-return 0;
-}
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
deleted file mode 100644
index 4685046..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSubScan.java
+++ /dev/null
@@ -1,119 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.Iterator;
-import java.util.List;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.physical.base.AbstractBase;
-import org.apache.drill.exec.physical.base.SubScan;
-import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.physical.base.PhysicalVisitor;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.ischema.SelectedTable;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonIgnore;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Iterators;
-
-public class MPJdbcSubScan extends AbstractBase implements SubScan {
-  private MPJdbcFormatPlugin plugin;
-  private MPJdbcFormatConfig pluginConfig;
-  private List<SchemaPath> columns;
-  private List<MPJdbcScanSpec> scanList;
-  private String userName;
-
-  @JsonCreator
-  public MPJdbcSubScan(@JacksonInject StoragePluginRegistry registry,
-      @JsonProperty("userName") String userName,
-      @JsonProperty("pluginConfig") MPJdbcFormatConfig pluginConfig,
-      @JsonProperty("ScanList") List<MPJdbcScanSpec> scanlist,
-      @JsonProperty("columns") List<SchemaPath> columns) throws ExecutionSetupException {
-    super(userName);
-    this.plugin = (MPJdbcFormatPlugin) registry.getPlugin(pluginConfig);
-    this.pluginConfig = pluginConfig;
-    this.columns = columns;
-    this.scanList = scanlist;
-    this.userName = userName;
-  }
-  public MPJdbcSubScan(MPJdbcFormatPlugin plugin,
-          @JsonProperty("userName") String userName,
-          @JsonProperty("pluginConfig") MPJdbcFormatConfig pluginConfig,
-          @JsonProperty("ScanList") List<MPJdbcScanSpec> scanlist,
-          @JsonProperty("columns") List<SchemaPath> columns) {
-    super(userName);
-    this.plugin = plugin;
-    this.pluginConfig = pluginConfig;
-    this.columns = columns;
-    this.scanList = scanlist;
-    this.userName = userName;
-    }
-
-  @Override
-  public int getOperatorType() {
-    return 55;
-  }
-
-  @JsonProperty("columns")
-  public List<SchemaPath> getColumns() {
-    // TODO Auto-generated method stub
-    return columns;
-  }
-
-  @JsonIgnore
-  public List<MPJdbcScanSpec> getScanList() {
-    return this.scanList;
-  }
-
-  @JsonIgnore
-  public MPJdbcFormatConfig getConfig() {
-    return this.pluginConfig;
-  }
-
-  @JsonIgnore
-  public MPJdbcFormatPlugin getPlugin() {
-    return this.plugin;
-  }
-
-  @Override
-  public boolean isExecutable() {
-    return false;
-  }
-
-  @Override
-  public <T, X, E extends Throwable> T accept(
-      PhysicalVisitor<T, X, E> physicalVisitor, X value) throws E {
-    return physicalVisitor.visitSubScan(this, value);
-  }
-
-  @Override
-  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
-    Preconditions.checkArgument(children.isEmpty());
-    return new MPJdbcSubScan(plugin,userName, pluginConfig, scanList, columns);
-  }
-
-  @Override
-  public Iterator<PhysicalOperator> iterator() {
-    return Iterators.emptyIterator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
deleted file mode 100755
index d408322..0000000
--- a/contrib/storage-mpjdbc/src/main/resources/bootstrap-storage-plugins.json
+++ /dev/null
@@ -1,12 +0,0 @@
-{
-  "storage":{
-    jdbc : {
-      type:"jdbc",
-      enabled: false,
-      driver:"",
-      uri:"jdbc://",
-      username:"",
-      password:""
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml b/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
deleted file mode 100644
index 0758908..0000000
--- a/contrib/storage-mpjdbc/src/main/resources/checkstyle-config.xml
+++ /dev/null
@@ -1,41 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!-- 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. -->
-<!DOCTYPE module PUBLIC
-    "-//Puppy Crawl//DTD Check Configuration 1.2//EN"
-    "http://www.puppycrawl.com/dtds/configuration_1_2.dtd">
-
-<!--  Checkstyle configuration for Apache Drill -->
-
-<module name="Checker">
-
-  <module name="TreeWalker">
-
-    <module name="IllegalImport">
-      <!-- For "org.apache.commons.lang.*" classes use the corresponding class from "org.apache.commons.lang3.*" -->
-      <property name="illegalPkgs" value="com.google.hive12,com.beust.jcommander.internal,jersey.repackaged,org.apache.commons.lang"/>
-    </module>
-
-    <module name="AvoidStarImport">
-      <property name="allowStaticMemberImports" value="true"/>
-    </module>
-
-    <module name="NeedBraces"/>
-
-  </module>
-
-  <module name="FileTabCharacter"/>
-
-  <module name="RegexpSingleline">
-    <property name="format" value="\s+$"/>
-  </module>
-
-</module>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml b/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
deleted file mode 100644
index 9d4682b..0000000
--- a/contrib/storage-mpjdbc/src/main/resources/checkstyle-suppressions.xml
+++ /dev/null
@@ -1,19 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!-- 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. -->
-<!DOCTYPE suppressions PUBLIC
-    "-//Puppy Crawl//DTD Suppressions 1.1//EN"
-    "suppressions_1_1.dtd">
-
-<!--  Checkstyle Suppressions for Apache Drill -->
-<suppressions>
-  <suppress files="[\\/]generated-sources[\\/]" checks="AvoidStarImport,NeedBraces"/>
-</suppressions>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/resources/drill-module.conf b/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
deleted file mode 100644
index e753703..0000000
--- a/contrib/storage-mpjdbc/src/main/resources/drill-module.conf
+++ /dev/null
@@ -1,30 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one or more
-// contributor license agreements.  See the NOTICE file distributed with
-// this work for additional information regarding copyright ownership.
-// The ASF licenses this file to You under the Apache License, Version 2.0
-// (the "License"); you may not use this file except in compliance with
-// the License.  You may obtain a copy of the License at
-//
-//     http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing, software
-// distributed under the License is distributed on an "AS IS" BASIS,
-// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-// See the License for the specific language governing permissions and
-// limitations under the License.
-//
-//  This file tells Drill to consider this module when class path scanning.
-//  This file can also include any supplementary configuration information.
-//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.
-
-drill.exec: {
-
-  sys.store.provider: {
-    jdbc : {
-      driver: "undefined",
-      uri : "jdbc://localhost",
-      username : "user",
-      password: "password"
-    }
-  }
-}


[06/15] drill git commit: DRILL-1666: Add test code for mongo storage plugin

Posted by ja...@apache.org.
DRILL-1666: Add test code for mongo storage plugin


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

Branch: refs/heads/master
Commit: 197d972fa42aa4cd2897d347a9bac58052a30c99
Parents: e12cd47
Author: akumarb2010 <ak...@gmail.com>
Authored: Mon Jul 6 22:35:25 2015 +0530
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 18:36:25 2015 -0700

----------------------------------------------------------------------
 contrib/storage-mongo/pom.xml                   |  13 +
 .../drill/exec/store/mongo/MongoTestBase.java   |  88 +++++++
 .../exec/store/mongo/MongoTestConstants.java    |  70 ++++++
 .../drill/exec/store/mongo/MongoTestSuit.java   | 245 +++++++++++++++++++
 .../store/mongo/TestMongoFilterPushDown.java    |  55 +++++
 .../store/mongo/TestMongoProjectPushDown.java   |  48 ++++
 .../exec/store/mongo/TestMongoQueries.java      |  55 +++++
 .../exec/store/mongo/TestTableGenerator.java    |  65 +++++
 .../src/test/resources/donuts.json              | 127 ++++++++++
 .../storage-mongo/src/test/resources/emp.json   |  19 ++
 10 files changed, 785 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/pom.xml b/contrib/storage-mongo/pom.xml
index 9a6a0b3..6efaf25 100644
--- a/contrib/storage-mongo/pom.xml
+++ b/contrib/storage-mongo/pom.xml
@@ -28,6 +28,10 @@
 
   <name>contrib/mongo-storage-plugin</name>
 
+  <properties>
+     <mongo.TestSuite>**/MongoTestSuit.class</mongo.TestSuite>
+  </properties>
+  
   <dependencies>
     <dependency>
       <groupId>org.apache.drill.exec</groupId>
@@ -63,6 +67,12 @@
       <version>2.1.1</version>
       <scope>test</scope>
     </dependency>
+    <dependency>
+      <groupId>de.flapdoodle.embed</groupId>
+      <artifactId>de.flapdoodle.embed.mongo</artifactId>
+      <version>1.48.0</version>
+      <scope>test</scope>
+    </dependency>
   </dependencies>
 
   <build>
@@ -71,6 +81,9 @@
         <groupId>org.apache.maven.plugins</groupId>
         <artifactId>maven-surefire-plugin</artifactId>
         <configuration>
+	  <includes>
+	    <include>${mongo.TestSuite}</include>
+	  </includes>
           <systemProperties>
             <property>
               <name>logback.log.dir</name>

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
new file mode 100644
index 0000000..cf3ca61
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestBase.java
@@ -0,0 +1,88 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.List;
+
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.rpc.user.QueryDataBatch;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+public class MongoTestBase extends PlanTestBase implements MongoTestConstants {
+  protected static MongoStoragePlugin storagePlugin;
+  protected static MongoStoragePluginConfig storagePluginConfig;
+
+  @BeforeClass
+  public static void setUpBeforeClass() throws Exception {
+    initMongoStoragePlugin();
+  }
+
+  public static void initMongoStoragePlugin() throws ExecutionSetupException {
+    final StoragePluginRegistry pluginRegistry = getDrillbitContext()
+        .getStorage();
+    storagePlugin = (MongoStoragePlugin) pluginRegistry
+        .getPlugin(MongoStoragePluginConfig.NAME);
+    storagePluginConfig = storagePlugin.getConfig();
+    storagePluginConfig.setEnabled(true);
+    pluginRegistry.createOrUpdate(MongoStoragePluginConfig.NAME,
+        storagePluginConfig, true);
+  }
+
+  public List<QueryDataBatch> runMongoSQLWithResults(String sql)
+      throws Exception {
+    return testSqlWithResults(sql);
+  }
+
+  public void runMongoSQLVerifyCount(String sql, int expectedRowCount)
+      throws Exception {
+    List<QueryDataBatch> results = runMongoSQLWithResults(sql);
+    printResultAndVerifyRowCount(results, expectedRowCount);
+  }
+
+  public void printResultAndVerifyRowCount(List<QueryDataBatch> results,
+      int expectedRowCount) throws SchemaChangeException {
+    int rowCount = printResult(results);
+    if (expectedRowCount != -1) {
+      Assert.assertEquals(expectedRowCount, rowCount);
+    }
+  }
+
+  public void testHelper(String query, String expectedExprInPlan,
+      int expectedRecordCount) throws Exception {
+    testPhysicalPlan(query, expectedExprInPlan);
+    int actualRecordCount = testSql(query);
+    assertEquals(
+        String.format(
+            "Received unexpected number of rows in output: expected=%d, received=%s",
+            expectedRecordCount, actualRecordCount), expectedRecordCount,
+        actualRecordCount);
+  }
+
+  @AfterClass
+  public static void tearDownMongoTestBase() throws Exception {
+    storagePlugin = null;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
new file mode 100644
index 0000000..d6d83d1
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
@@ -0,0 +1,70 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+public interface MongoTestConstants {
+
+  public static final String LOCALHOST = "localhost";
+  public static final int CONFIG_SERVER_PORT = 27019;
+  public static final int MONGOD_1_PORT = 27020;
+  public static final int MONGOD_2_PORT = 27021;
+  public static final int MONGOD_3_PORT = 27022;
+
+  public static final int MONGOD_4_PORT = 27023;
+  public static final int MONGOD_5_PORT = 27024;
+  public static final int MONGOD_6_PORT = 27025;
+
+  public static final int MONGOS_PORT = 27017;
+
+  public static final String CONFIG_DB = "config";
+  public static final String ADMIN_DB = "admin";
+
+  public static final String TEST_DB = "testDB";
+  public static final String EMPLOYEE_DB = "employee";
+
+  public static final String DONUTS_COLLECTION = "donuts";
+  public static final String EMPINFO_COLLECTION = "empinfo";
+
+  public static final String DONUTS_DATA = "donuts.json";
+  public static final String EMP_DATA = "emp.json";
+
+  public static final String REPLICA_SET_1_NAME = "shard_1_replicas";
+  public static final String REPLICA_SET_2_NAME = "shard_2_replicas";
+
+  // test queries
+  public static final String TEST_QUERY_1 = "SELECT * FROM mongo.employee.`empinfo` limit 5";
+  public static final String TEST_QUERY_LIMIT = "SELECT first_name, last_name FROM mongo.employee.`empinfo` limit 2;";
+
+  // test query template1
+  public static final String TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s`";
+  public static final String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2 = "select `employee_id`, `rating` from mongo.%s.`%s`";
+  public static final String TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3 = "select * from mongo.%s.`%s`";
+  public static final String TEST_FILTER_PUSH_DOWN_IS_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is null";
+  public static final String TEST_FILTER_PUSH_DOWN_IS_NOT_NULL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where position_id is not null";
+  public static final String TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating = 52.17";
+  public static final String TEST_FILTER_PUSH_DOWN_NOT_EQUAL_QUERY_TEMPLATE_1 = "SELECT `employee_id` FROM mongo.%s.`%s` where rating != 52.17";
+  public static final String TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating < 52.17";
+  public static final String TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1 = "SELECT `full_name` FROM mongo.%s.`%s` where rating > 52.17";
+  public static final String TEST_EMPTY_TABLE_QUERY_TEMPLATE = "select count(*) from mongo.%s.`%s`";
+
+  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1 = "select `employee_id` from mongo.%s.`%s` where isFTE = true";
+  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2 = "select `employee_id` from mongo.%s.`%s` where isFTE = false";
+  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3 = "select `employee_id` from mongo.%s.`%s` where position_id = 16 and isFTE = true";
+  public static final String TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4 = "select `employee_id` from mongo.%s.`%s` where (position_id = 16 and isFTE = true) or last_name = 'Yonce'";
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
new file mode 100644
index 0000000..c52df63
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuit.java
@@ -0,0 +1,245 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.bson.Document;
+import org.bson.conversions.Bson;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.runner.RunWith;
+import org.junit.runners.Suite;
+import org.junit.runners.Suite.SuiteClasses;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.mongodb.MongoClient;
+import com.mongodb.ServerAddress;
+import com.mongodb.client.MongoCollection;
+import com.mongodb.client.MongoDatabase;
+import com.mongodb.client.model.IndexOptions;
+
+import de.flapdoodle.embed.mongo.Command;
+import de.flapdoodle.embed.mongo.MongodExecutable;
+import de.flapdoodle.embed.mongo.MongodProcess;
+import de.flapdoodle.embed.mongo.MongodStarter;
+import de.flapdoodle.embed.mongo.config.IMongoCmdOptions;
+import de.flapdoodle.embed.mongo.config.IMongodConfig;
+import de.flapdoodle.embed.mongo.config.IMongosConfig;
+import de.flapdoodle.embed.mongo.config.MongoCmdOptionsBuilder;
+import de.flapdoodle.embed.mongo.config.MongodConfigBuilder;
+import de.flapdoodle.embed.mongo.config.MongosConfigBuilder;
+import de.flapdoodle.embed.mongo.config.Net;
+import de.flapdoodle.embed.mongo.config.RuntimeConfigBuilder;
+import de.flapdoodle.embed.mongo.config.Storage;
+import de.flapdoodle.embed.mongo.distribution.Version;
+import de.flapdoodle.embed.mongo.tests.MongosSystemForTestFactory;
+import de.flapdoodle.embed.process.config.IRuntimeConfig;
+import de.flapdoodle.embed.process.runtime.Network;
+
+@RunWith(Suite.class)
+@SuiteClasses({ TestMongoFilterPushDown.class, TestMongoProjectPushDown.class,
+    TestMongoQueries.class, TestMongoChunkAssignment.class })
+public class MongoTestSuit implements MongoTestConstants {
+
+  private static final Logger logger = LoggerFactory
+      .getLogger(MongoTestSuit.class);
+
+  protected static MongoClient mongoClient;
+
+  private static boolean distMode = System.getProperty(
+      "drill.mongo.tests.shardMode", "true").equalsIgnoreCase("true");
+  private static boolean authEnabled = System.getProperty(
+      "drill.mongo.tests.authEnabled", "false").equalsIgnoreCase("true");
+
+  private static class DistributedMode {
+    private static MongosSystemForTestFactory mongosTestFactory;
+
+    private static void setup() throws Exception {
+      // creating configServers
+      List<IMongodConfig> configServers = new ArrayList<IMongodConfig>(1);
+      IMongodConfig configIMongodConfig = crateConfigServerConfig(
+          CONFIG_SERVER_PORT, true);
+      configServers.add(configIMongodConfig);
+
+      // creating replicaSets
+      Map<String, List<IMongodConfig>> replicaSets = new HashMap<String, List<IMongodConfig>>();
+      List<IMongodConfig> replicaSet1 = new ArrayList<IMongodConfig>();
+      replicaSet1.add(crateIMongodConfig(MONGOD_1_PORT, false,
+          REPLICA_SET_1_NAME));
+      replicaSet1.add(crateIMongodConfig(MONGOD_2_PORT, false,
+          REPLICA_SET_1_NAME));
+      replicaSet1.add(crateIMongodConfig(MONGOD_3_PORT, false,
+          REPLICA_SET_1_NAME));
+      replicaSets.put(REPLICA_SET_1_NAME, replicaSet1);
+      List<IMongodConfig> replicaSet2 = new ArrayList<IMongodConfig>();
+      replicaSet2.add(crateIMongodConfig(MONGOD_4_PORT, false,
+          REPLICA_SET_2_NAME));
+      replicaSet2.add(crateIMongodConfig(MONGOD_5_PORT, false,
+          REPLICA_SET_2_NAME));
+      replicaSet2.add(crateIMongodConfig(MONGOD_6_PORT, false,
+          REPLICA_SET_2_NAME));
+      replicaSets.put(REPLICA_SET_2_NAME, replicaSet2);
+
+      // create mongos
+      IMongosConfig mongosConfig = createIMongosConfig();
+      mongosTestFactory = new MongosSystemForTestFactory(mongosConfig,
+          replicaSets, configServers, EMPLOYEE_DB, EMPINFO_COLLECTION,
+          "employee_id");
+
+      try {
+        mongosTestFactory.start();
+        mongoClient = (MongoClient) mongosTestFactory.getMongo();
+      } catch (Throwable e) {
+        logger.error(" Error while starting shrded cluster. ", e);
+        throw new Exception(" Error while starting shrded cluster. ", e);
+      }
+    }
+
+    private static IMongodConfig crateConfigServerConfig(int configServerPort,
+        boolean flag) throws UnknownHostException, IOException {
+      IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder().verbose(false)
+          .build();
+
+      IMongodConfig mongodConfig = new MongodConfigBuilder()
+          .version(Version.Main.PRODUCTION)
+          .net(new Net(LOCALHOST, configServerPort, Network.localhostIsIPv6()))
+          .configServer(flag).cmdOptions(cmdOptions).build();
+      return mongodConfig;
+    }
+
+    private static IMongodConfig crateIMongodConfig(int mongodPort,
+        boolean flag, String replicaName) throws UnknownHostException,
+        IOException {
+      IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder().verbose(false)
+          .build();
+
+      Storage replication = new Storage(null, replicaName, 0);
+      IMongodConfig mongodConfig = new MongodConfigBuilder()
+          .version(Version.Main.PRODUCTION)
+          .net(new Net(LOCALHOST, mongodPort, Network.localhostIsIPv6()))
+          .configServer(flag).replication(replication).cmdOptions(cmdOptions)
+          .build();
+      return mongodConfig;
+    }
+
+    private static IMongosConfig createIMongosConfig()
+        throws UnknownHostException, IOException {
+      IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder().verbose(false)
+          .build();
+
+      IMongosConfig mongosConfig = new MongosConfigBuilder()
+          .version(Version.Main.PRODUCTION)
+          .net(new Net(LOCALHOST, MONGOS_PORT, Network.localhostIsIPv6()))
+          .configDB(LOCALHOST + ":" + CONFIG_SERVER_PORT)
+          .cmdOptions(cmdOptions).build();
+      return mongosConfig;
+    }
+
+    private static void cleanup() {
+      if (mongosTestFactory != null) {
+        mongosTestFactory.stop();
+      }
+    }
+
+  }
+
+  private static class SingleMode {
+
+    private static MongodExecutable mongodExecutable;
+    private static MongodProcess mongod;
+
+    private static void setup() throws UnknownHostException, IOException {
+      IMongoCmdOptions cmdOptions = new MongoCmdOptionsBuilder().verbose(false)
+          .enableAuth(authEnabled).build();
+
+      IMongodConfig mongodConfig = new MongodConfigBuilder()
+          .version(Version.Main.PRODUCTION)
+          .net(new Net(LOCALHOST, MONGOS_PORT, Network.localhostIsIPv6()))
+          .cmdOptions(cmdOptions).build();
+
+      IRuntimeConfig runtimeConfig = new RuntimeConfigBuilder().defaults(
+          Command.MongoD).build();
+      mongodExecutable = MongodStarter.getInstance(runtimeConfig).prepare(
+          mongodConfig);
+      mongod = mongodExecutable.start();
+      mongoClient = new MongoClient(new ServerAddress(LOCALHOST, MONGOS_PORT));
+      createDbAndCollections(EMPLOYEE_DB, EMPINFO_COLLECTION, "employee_id");
+    }
+
+    private static void cleanup() {
+      if (mongod != null) {
+        mongod.stop();
+      }
+      if (mongodExecutable != null) {
+        mongodExecutable.cleanup();
+      }
+    }
+  }
+
+  @BeforeClass
+  public static void initMongo() throws Exception {
+    synchronized (MongoTestSuit.class) {
+      if (distMode) {
+        logger.info("Executing tests in distributed mode");
+        DistributedMode.setup();
+      } else {
+        logger.info("Executing tests in single mode");
+        SingleMode.setup();
+      }
+      TestTableGenerator.importData(EMPLOYEE_DB, EMPINFO_COLLECTION, EMP_DATA);
+    }
+  }
+
+  private static void createDbAndCollections(String dbName,
+      String collectionName, String indexFieldName) {
+    MongoDatabase db = mongoClient.getDatabase(dbName);
+    MongoCollection<Document> mongoCollection = db
+        .getCollection(collectionName);
+    if (mongoCollection == null) {
+      db.createCollection(collectionName);
+      mongoCollection = db.getCollection(collectionName);
+    }
+    IndexOptions indexOptions = new IndexOptions().unique(true)
+        .background(false).name(indexFieldName);
+    Bson keys = new Document(indexFieldName, Integer.valueOf(1));
+    mongoCollection.createIndex(keys, indexOptions);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() throws Exception {
+    if (mongoClient != null) {
+      mongoClient.dropDatabase(TEST_DB);
+      mongoClient.dropDatabase(EMPLOYEE_DB);
+      mongoClient.close();
+    }
+    synchronized (MongoTestSuit.class) {
+      if (distMode) {
+        DistributedMode.cleanup();
+      } else {
+        SingleMode.cleanup();
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
new file mode 100644
index 0000000..633d0a9
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoFilterPushDown.java
@@ -0,0 +1,55 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import org.junit.Test;
+
+public class TestMongoFilterPushDown extends MongoTestBase {
+
+  @Test
+  public void testFilterPushDownIsEqual() throws Exception {
+    String queryString = String.format(
+        TEST_FILTER_PUSH_DOWN_EQUAL_QUERY_TEMPLATE_1, EMPLOYEE_DB,
+        EMPINFO_COLLECTION);
+    String expectedExpr = "\"$eq\" : 52.17";
+    testHelper(queryString, expectedExpr, 1);
+  }
+
+  @Test
+  public void testFilterPushDownLessThanWithSingleField() throws Exception {
+    String queryString = String.format(
+        TEST_FILTER_PUSH_DOWN_LESS_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
+        EMPINFO_COLLECTION);
+    System.out.println("testFilterPushDownLessThanWithSingleField Query: "
+        + queryString);
+    String expectedExpr = "\"$lt\" : 52.17";
+    testHelper(queryString, expectedExpr, 9);
+  }
+
+  @Test
+  public void testFilterPushDownGreaterThanWithSingleField() throws Exception {
+    String queryString = String.format(
+        TEST_FILTER_PUSH_DOWN_GREATER_THAN_QUERY_TEMPLATE_1, EMPLOYEE_DB,
+        EMPINFO_COLLECTION);
+    System.out.println("testFilterPushDownGreaterThanWithSingleField Query: "
+        + queryString);
+    String expectedExpr = "\"$gt\" : 52.17";
+    testHelper(queryString, expectedExpr, 9);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
new file mode 100644
index 0000000..54ace3f
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoProjectPushDown.java
@@ -0,0 +1,48 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import org.junit.Test;
+
+public class TestMongoProjectPushDown extends MongoTestBase {
+
+  @Test
+  public void testSingleColumnProject() throws Exception {
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN_TEMPLATE_1,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    String expectedColNames = " \"columns\" : [ \"`employee_id`\" ]";
+    testHelper(query, expectedColNames, 19);
+  }
+
+  @Test
+  public void testMultipleColumnsProject() throws Exception {
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_2,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    String expectedColNames = "\"columns\" : [ \"`employee_id`\", \"`rating`\" ]";
+    testHelper(query, expectedColNames, 19);
+  }
+
+  @Test
+  public void testStarProject() throws Exception {
+    String query = String.format(TEST_QUERY_PROJECT_PUSH_DOWN__TEMPLATE_3,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    String expectedColNames = "*";
+    testHelper(query, expectedColNames, 19);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
new file mode 100644
index 0000000..f7e502a
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoQueries.java
@@ -0,0 +1,55 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import org.junit.Test;
+
+public class TestMongoQueries extends MongoTestBase {
+
+  @Test
+  public void testBooleanFilter() throws Exception {
+    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE1,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    runMongoSQLVerifyCount(queryString, 11);
+    queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE2,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    runMongoSQLVerifyCount(queryString, 8);
+  }
+
+  @Test
+  public void testWithANDOperator() throws Exception {
+    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    runMongoSQLVerifyCount(queryString, 4);
+  }
+
+  @Test
+  public void testWithOROperator() throws Exception {
+    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE3,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    runMongoSQLVerifyCount(queryString, 4);
+  }
+
+  @Test
+  public void testResultCount() throws Exception {
+    String queryString = String.format(TEST_BOOLEAN_FILTER_QUERY_TEMPLATE4,
+        EMPLOYEE_DB, EMPINFO_COLLECTION);
+    runMongoSQLVerifyCount(queryString, 5);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
new file mode 100644
index 0000000..62e1204
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestTableGenerator.java
@@ -0,0 +1,65 @@
+/**
+ * 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.drill.exec.store.mongo;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+
+import org.apache.commons.lang3.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.io.Resources;
+
+import de.flapdoodle.embed.mongo.MongoImportExecutable;
+import de.flapdoodle.embed.mongo.MongoImportStarter;
+import de.flapdoodle.embed.mongo.config.IMongoImportConfig;
+import de.flapdoodle.embed.mongo.config.MongoImportConfigBuilder;
+import de.flapdoodle.embed.mongo.config.Net;
+import de.flapdoodle.embed.mongo.distribution.Version;
+import de.flapdoodle.embed.process.runtime.Network;
+
+public class TestTableGenerator implements MongoTestConstants {
+
+  private static final Logger logger = LoggerFactory
+      .getLogger(TestTableGenerator.class);
+
+  public static void importData(String dbName, String collectionName,
+      String fileName) throws IOException {
+    String jsonFile = Resources.getResource(fileName).toString();
+    jsonFile = jsonFile.replaceFirst("file:", StringUtils.EMPTY);
+    generateTable(dbName, collectionName, jsonFile, true, true, false);
+  }
+
+  public static void generateTable(String dbName, String collection,
+      String jsonFile, Boolean jsonArray, Boolean upsert, Boolean drop)
+      throws UnknownHostException, IOException {
+    logger.info("Started importing file {} into collection {} ", jsonFile,
+        collection);
+    IMongoImportConfig mongoImportConfig = new MongoImportConfigBuilder()
+        .version(Version.Main.PRODUCTION)
+        .net(new Net(MONGOS_PORT, Network.localhostIsIPv6())).db(dbName)
+        .collection(collection).upsert(upsert).dropCollection(drop)
+        .jsonArray(jsonArray).importFile(jsonFile).build();
+    MongoImportExecutable importExecutable = MongoImportStarter
+        .getDefaultInstance().prepare(mongoImportConfig);
+    importExecutable.start();
+    logger.info("Imported file {} into collection {} ", jsonFile, collection);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/resources/donuts.json
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/resources/donuts.json b/contrib/storage-mongo/src/test/resources/donuts.json
new file mode 100644
index 0000000..45beeb8
--- /dev/null
+++ b/contrib/storage-mongo/src/test/resources/donuts.json
@@ -0,0 +1,127 @@
+[
+{
+    "id": "0001",
+    "type": "donut",
+    "name": "Cake",
+    "ppu": 0.55,
+    "sales": 35,
+
+    "batters":
+      {
+        "batter":
+          [
+            { "id": "1001", "type": "Regular" },
+            { "id": "1002", "type": "Chocolate" },
+            { "id": "1003", "type": "Blueberry" },
+            { "id": "1004", "type": "Devil's Food" }
+          ]
+      },
+    "topping":
+      [
+        { "id": "5001", "type": "None" },
+        { "id": "5002", "type": "Glazed" },
+        { "id": "5005", "type": "Sugar" },
+        { "id": "5007", "type": "Powdered Sugar" },
+        { "id": "5006", "type": "Chocolate with Sprinkles" },
+        { "id": "5003", "type": "Chocolate" },
+        { "id": "5004", "type": "Maple" }
+      ]
+},
+{
+    "id": "0002",
+    "type": "donut",
+    "name": "Raised",
+    "ppu": 0.69,
+    "sales": 145,
+    "batters":
+      {
+        "batter":
+          [
+            { "id": "1001", "type": "Regular" }
+          ]
+      },
+    "topping":
+      [
+        { "id": "5001", "type": "None" },
+        { "id": "5002", "type": "Glazed" },
+        { "id": "5005", "type": "Sugar" },
+        { "id": "5003", "type": "Chocolate" },
+        { "id": "5004", "type": "Maple" }
+      ]
+},
+{
+    "id": "0003",
+    "type": "donut",
+    "name": "Old Fashioned",
+    "ppu": 0.55,
+    "sales": 300,
+    "batters":
+      {
+        "batter":
+          [
+            { "id": "1001", "type": "Regular" },
+            { "id": "1002", "type": "Chocolate" }
+          ]
+      },
+    "topping":
+      [
+        { "id": "5001", "type": "None" },
+        { "id": "5002", "type": "Glazed" },
+        { "id": "5003", "type": "Chocolate" },
+        { "id": "5004", "type": "Maple" }
+      ]
+},
+{
+    "id": "0004",
+    "type": "donut",
+    "name": "Filled",
+    "ppu": 0.69,
+    "sales": 14,
+
+    "batters":
+      {
+        "batter":
+          [
+            { "id": "1001", "type": "Regular" },
+            { "id": "1002", "type": "Chocolate" },
+            { "id": "1003", "type": "Blueberry" },
+            { "id": "1004", "type": "Devil's Food" }
+          ]
+      },
+    "topping":
+      [
+        { "id": "5001", "type": "None" },
+        { "id": "5002", "type": "Glazed" },
+        { "id": "5005", "type": "Sugar" },
+        { "id": "5007", "type": "Powdered Sugar" },
+        { "id": "5006", "type": "Chocolate with Sprinkles" },
+        { "id": "5003", "type": "Chocolate" },
+        { "id": "5004", "type": "Maple" }
+      ],
+    "filling":
+      [
+        { "id": "6001", "type": "None" },
+        { "id": "6002", "type": "Raspberry" },
+        { "id": "6003", "type": "Lemon" },
+        { "id": "6004", "type": "Chocolate" },
+        { "id": "6005", "type": "Kreme" }
+      ]
+},
+{
+    "id": "0005",
+    "type": "donut",
+    "name": "Apple Fritter",
+    "sales": 700,
+    "batters":
+      {
+        "batter":
+          [
+            { "id": "1001", "type": "Regular" }
+          ]
+      },
+    "topping":
+      [
+        { "id": "5002", "type": "Glazed" }
+      ]
+}
+]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/197d972f/contrib/storage-mongo/src/test/resources/emp.json
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/test/resources/emp.json b/contrib/storage-mongo/src/test/resources/emp.json
new file mode 100644
index 0000000..5b786d5
--- /dev/null
+++ b/contrib/storage-mongo/src/test/resources/emp.json
@@ -0,0 +1,19 @@
+[{"employee_id":1101,"full_name":"Steve Eurich","first_name":"Steve","last_name":"Eurich","position_id":16,"rating":23.0,"position":"Store T","isFTE":true},
+{"employee_id":1102,"full_name":"Mary Pierson","first_name":"Mary","last_name":"Pierson","position_id":16,"rating":45.6,"position":"Store T","isFTE":true},
+{"employee_id":1103,"full_name":"Leo Jones","first_name":"Leo","last_name":"Jones","position_id":16,"position":"Store Tem","rating":85.94,"isFTE":true},
+{"employee_id":1104,"full_name":"Nancy Beatty","first_name":"Nancy","last_name":"Beatty","position_id":16,"position":"Store T","rating":97.16,"isFTE":false},
+{"employee_id":1105,"full_name":"Clara McNight","first_name":"Clara","last_name":"McNight","position_id":16,"position":"Store","rating":81.25,"isFTE":true},
+{"employee_id":1106,"first_name":"Marcella","last_name":"Isaacs","position_id":17,"position":"Stor","rating":67.86,"isFTE":false},
+{"employee_id":1107,"full_name":"Charlotte Yonce","first_name":"Charlotte","last_name":"Yonce","position_id":17,"position":"Stor","rating":52.17,"isFTE":true},
+{"employee_id":1108,"full_name":"Benjamin Foster","first_name":"Benjamin","last_name":"Foster","position_id":17,"position":"Stor","rating":89.8,"isFTE":false},
+{"employee_id":1109,"full_name":"John Reed","first_name":"John","last_name":"Reed","position_id":17,"position":"Store Per","rating":12.9,"isFTE":false},
+{"employee_id":1110,"full_name":"Lynn Kwiatkowski","first_name":"Lynn","last_name":"Kwiatkowski","position_id":17,"position":"St","rating":25.76,"isFTE":true},
+{"employee_id":1111,"full_name":"Donald Vann","first_name":"Donald","last_name":"Vann","position_id":17,"position":"Store Per","rating":34.86,"isFTE":false},
+{"employee_id":1112,"first_name":"William","last_name":"Smith","position":"St","rating":79.06,"isFTE":true},
+{"employee_id":1113,"full_name":"Amy Hensley","first_name":"Amy","last_name":"Hensley","position_id":17,"position":"Store Pe","rating":82.96,"isFTE":false},
+{"employee_id":1114,"full_name":"Judy Owens","first_name":"Judy","last_name":"Owens","position_id":17,"position":"Store Per","rating":24.6,"isFTE":true},
+{"employee_id":1115,"full_name":"Frederick Castillo","first_name":"Frederick","last_name":"Castillo","position_id":17,"position":"S","rating":82.36,"isFTE":false},
+{"employee_id":1116,"full_name":"Phil Munoz","first_name":"Phil","last_name":"Munoz","position_id":17,"position":"Store Per","rating":97.63,"isFTE":false},
+{"employee_id":1117,"full_name":"Lori Lightfoot","first_name":"Lori","last_name":"Lightfoot","position_id":17,"position":"Store","rating":39.16,"isFTE":true},
+{"employee_id":1,"full_name":"Kumar","first_name":"Anil","last_name":"B","position_id":19,"position":"Store","rating":45.45,"isFTE":true},
+{"employee_id":2,"full_name":"Kamesh","first_name":"Bh","last_name":"Venkata","position":"Store","rating":32.89,"isFTE":true}]


[05/15] drill git commit: DRILL-3180: JDBC Storage Plugin updates.

Posted by ja...@apache.org.
DRILL-3180: JDBC Storage Plugin updates.

- Move to leverage Calcite's JDBC adapter capabilities for pushdowns, schema, etc.
- Add test cases using Derby


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

Branch: refs/heads/master
Commit: e12cd470e4ab57b025840fdfa200a051a01df029
Parents: 8478e9f
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sat Aug 1 18:11:51 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 18:26:44 2015 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/AutoCloseables.java |   6 +-
 contrib/pom.xml                                 |   1 +
 contrib/storage-jdbc/pom.xml                    |  76 +++
 .../exec/store/jdbc/DrillJdbcRuleBase.java      | 119 +++++
 .../drill/exec/store/jdbc/JdbcBatchCreator.java |  42 ++
 .../apache/drill/exec/store/jdbc/JdbcDrel.java  |  51 ++
 .../exec/store/jdbc/JdbcExpressionCheck.java    | 119 +++++
 .../drill/exec/store/jdbc/JdbcGroupScan.java    | 107 +++++
 .../exec/store/jdbc/JdbcIntermediatePrel.java   |  79 ++++
 .../apache/drill/exec/store/jdbc/JdbcPrel.java  | 120 +++++
 .../drill/exec/store/jdbc/JdbcRecordReader.java | 431 +++++++++++++++++
 .../exec/store/jdbc/JdbcStorageConfig.java      | 120 +++++
 .../exec/store/jdbc/JdbcStoragePlugin.java      | 400 ++++++++++++++++
 .../drill/exec/store/jdbc/JdbcSubScan.java      |  71 +++
 .../resources/bootstrap-storage-plugins.json    |  10 +
 .../src/main/resources/drill-module.conf        |  18 +
 .../drill/exec/store/jdbc/TestJdbcPlugin.java   | 181 +++++++
 .../resources/bootstrap-storage-plugins.json    |  10 +
 .../storage-jdbc/src/test/resources/logback.xml |  48 ++
 contrib/storage-mpjdbc/pom.xml                  |  79 ----
 .../exec/store/mpjdbc/MPJdbcBatchCreator.java   |  54 ---
 .../drill/exec/store/mpjdbc/MPJdbcClient.java   | 300 ------------
 .../exec/store/mpjdbc/MPJdbcClientOptions.java  |  52 --
 .../exec/store/mpjdbc/MPJdbcCnxnManager.java    |  69 ---
 .../exec/store/mpjdbc/MPJdbcFilterBuilder.java  | 235 ---------
 .../exec/store/mpjdbc/MPJdbcFilterRule.java     |  60 ---
 .../exec/store/mpjdbc/MPJdbcFormatConfig.java   | 109 -----
 .../exec/store/mpjdbc/MPJdbcFormatPlugin.java   | 170 -------
 .../exec/store/mpjdbc/MPJdbcGroupScan.java      | 181 -------
 .../exec/store/mpjdbc/MPJdbcRecordReader.java   | 471 -------------------
 .../drill/exec/store/mpjdbc/MPJdbcScanSpec.java |  76 ---
 .../exec/store/mpjdbc/MPJdbcSchemaConfig.java   |  80 ----
 .../exec/store/mpjdbc/MPJdbcSchemaFilter.java   |  23 -
 .../exec/store/mpjdbc/MPJdbcSchemaSubScan.java  |  55 ---
 .../drill/exec/store/mpjdbc/MPJdbcSubScan.java  | 119 -----
 .../resources/bootstrap-storage-plugins.json    |  12 -
 .../src/main/resources/checkstyle-config.xml    |  41 --
 .../main/resources/checkstyle-suppressions.xml  |  19 -
 .../src/main/resources/drill-module.conf        |  30 --
 .../exec/planner/logical/DrillJoinRel.java      |  26 +-
 .../exec/planner/logical/DrillRuleSets.java     |  11 +-
 .../physical/DrillDistributionTraitDef.java     |   7 +-
 .../physical/explain/NumberingRelWriter.java    |  18 +-
 .../drill/exec/planner/sql/SchemaUtilites.java  |   2 +-
 .../planner/sql/handlers/DefaultSqlHandler.java |  13 +-
 .../planner/sql/handlers/PrelFinalizable.java   |  47 ++
 .../apache/drill/exec/store/AbstractSchema.java |   6 +-
 .../drill/exec/store/AbstractStoragePlugin.java |   3 +-
 .../apache/drill/exec/store/StoragePlugin.java  |   3 +-
 .../drill/exec/store/StoragePluginRegistry.java |   8 +-
 .../drill/exec/store/SubSchemaWrapper.java      |  15 +-
 51 files changed, 2115 insertions(+), 2288 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/common/src/main/java/org/apache/drill/common/AutoCloseables.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/AutoCloseables.java b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
index c080c52..3c4aa23 100644
--- a/common/src/main/java/org/apache/drill/common/AutoCloseables.java
+++ b/common/src/main/java/org/apache/drill/common/AutoCloseables.java
@@ -36,10 +36,14 @@ public class AutoCloseables {
    * @param logger the logger to use to record the exception if there was one
    */
   public static void close(final AutoCloseable ac, final Logger logger) {
+    if (ac == null) {
+      return;
+    }
+
     try {
       ac.close();
     } catch(Exception e) {
-      logger.warn("Failure on close(): " + e);
+      logger.warn("Failure on close(): {}", e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/pom.xml b/contrib/pom.xml
index 8c00e76..2f3ac9f 100644
--- a/contrib/pom.xml
+++ b/contrib/pom.xml
@@ -35,6 +35,7 @@
     <module>storage-hbase</module>
     <module>storage-hive</module>
     <module>storage-mongo</module>
+    <module>storage-jdbc</module>
     <module>sqlline</module>
     <module>data</module>
   </modules>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/pom.xml b/contrib/storage-jdbc/pom.xml
new file mode 100755
index 0000000..be20811
--- /dev/null
+++ b/contrib/storage-jdbc/pom.xml
@@ -0,0 +1,76 @@
+<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>drill-contrib-parent</artifactId>
+    <groupId>org.apache.drill.contrib</groupId>
+    <version>1.2.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>drill-jdbc-storage</artifactId>
+
+  <name>contrib/jdbc-storage-plugin</name>
+
+  <dependencies>
+
+
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    
+    <!-- Test dependencies -->
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.yammer.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>2.1.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derbyclient</artifactId>
+      <version>10.11.1.1</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.derby</groupId>
+      <artifactId>derbynet</artifactId>
+      <version>10.11.1.1</version>
+      <scope>test</scope>
+    </dependency>    
+  </dependencies>
+
+  
+</project>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
new file mode 100644
index 0000000..bbb4daf
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/DrillJdbcRuleBase.java
@@ -0,0 +1,119 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.calcite.adapter.jdbc.JdbcConvention;
+import org.apache.calcite.adapter.jdbc.JdbcRules;
+import org.apache.calcite.plan.Convention;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelTrait;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.logical.LogicalFilter;
+import org.apache.calcite.rel.logical.LogicalProject;
+import org.apache.calcite.rex.RexNode;
+
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.CacheLoader;
+import com.google.common.cache.LoadingCache;
+
+abstract class DrillJdbcRuleBase extends ConverterRule {
+
+  protected final LoadingCache<RexNode, Boolean> checkedExpressions = CacheBuilder.newBuilder()
+      .maximumSize(1000)
+      .expireAfterWrite(10, TimeUnit.MINUTES)
+      .build(
+          new CacheLoader<RexNode, Boolean>() {
+            public Boolean load(RexNode expr) {
+              return JdbcExpressionCheck.isOnlyStandardExpressions(expr);
+            }
+          });
+
+  protected final JdbcConvention out;
+
+  private DrillJdbcRuleBase(Class<? extends RelNode> clazz, RelTrait in, JdbcConvention out, String description) {
+    super(clazz, in, out, description);
+    this.out = out;
+  }
+
+  static class DrillJdbcProjectRule extends DrillJdbcRuleBase {
+
+    public DrillJdbcProjectRule(JdbcConvention out) {
+      super(LogicalProject.class, Convention.NONE, out, "JdbcProjectRule");
+    }
+
+    public RelNode convert(RelNode rel) {
+      LogicalProject project = (LogicalProject) rel;
+      return new JdbcRules.JdbcProject(rel.getCluster(), rel.getTraitSet().replace(this.out), convert(
+          project.getInput(), project.getInput().getTraitSet().replace(this.out)), project.getProjects(),
+          project.getRowType());
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      try {
+
+        final LogicalProject project = (LogicalProject) call.rel(0);
+        for (RexNode node : project.getChildExps()) {
+          if (!checkedExpressions.get(node)) {
+            return false;
+          }
+        }
+        return true;
+
+      } catch (ExecutionException e) {
+        throw new IllegalStateException("Failure while trying to evaluate pushdown.", e);
+      }
+    }
+  }
+
+  static class DrillJdbcFilterRule extends DrillJdbcRuleBase {
+
+    public DrillJdbcFilterRule(JdbcConvention out) {
+      super(LogicalFilter.class, Convention.NONE, out, "DrillJdbcFilterRule");
+    }
+
+    public RelNode convert(RelNode rel) {
+      LogicalFilter filter = (LogicalFilter) rel;
+
+      return new JdbcRules.JdbcFilter(rel.getCluster(), rel.getTraitSet().replace(this.out), convert(filter.getInput(),
+          filter.getInput().getTraitSet().replace(this.out)), filter.getCondition());
+    }
+
+    @Override
+    public boolean matches(RelOptRuleCall call) {
+      try {
+
+        final LogicalFilter filter = (LogicalFilter) call.rel(0);
+        for (RexNode node : filter.getChildExps()) {
+          if (!checkedExpressions.get(node)) {
+            return false;
+          }
+        }
+        return true;
+
+      } catch (ExecutionException e) {
+        throw new IllegalStateException("Failure while trying to evaluate pushdown.", e);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcBatchCreator.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcBatchCreator.java
new file mode 100755
index 0000000..fa44b55
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcBatchCreator.java
@@ -0,0 +1,42 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+//import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+
+import com.google.common.base.Preconditions;
+
+public class JdbcBatchCreator implements BatchCreator<JdbcSubScan> {
+  @Override
+  public ScanBatch getBatch(FragmentContext context, JdbcSubScan config,
+      List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    JdbcStoragePlugin plugin = config.getPlugin();
+    RecordReader reader = new JdbcRecordReader(context, plugin.getSource(), config.getSql(), plugin.getName());
+    return new ScanBatch(config, context, Collections.singletonList(reader).iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcDrel.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcDrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcDrel.java
new file mode 100644
index 0000000..52dd29f
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcDrel.java
@@ -0,0 +1,51 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.SingleRel;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
+
+public class JdbcDrel extends SingleRel implements DrillRel {
+
+  public JdbcDrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new JdbcDrel(getCluster(), traitSet, inputs.iterator().next());
+  }
+
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return copy(getTraitSet(), getInputs());
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    throw new UnsupportedOperationException();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
new file mode 100644
index 0000000..2015a77
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcExpressionCheck.java
@@ -0,0 +1,119 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexCorrelVariable;
+import org.apache.calcite.rex.RexDynamicParam;
+import org.apache.calcite.rex.RexFieldAccess;
+import org.apache.calcite.rex.RexFieldCollation;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexLiteral;
+import org.apache.calcite.rex.RexLocalRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.rex.RexOver;
+import org.apache.calcite.rex.RexRangeRef;
+import org.apache.calcite.rex.RexVisitor;
+import org.apache.calcite.rex.RexWindow;
+import org.apache.drill.exec.planner.sql.DrillSqlOperator;
+
+/**
+ * Visitor class that determines whether or not a particular RexNode expression tree contains only standard expressions.
+ * If RexNode tree contains Drill specific expressions, the tree will respond with false.
+ */
+class JdbcExpressionCheck implements RexVisitor<Boolean> {
+
+  private static final JdbcExpressionCheck INSTANCE = new JdbcExpressionCheck();
+
+  public static boolean isOnlyStandardExpressions(RexNode rex) {
+    return rex.accept(INSTANCE);
+  }
+
+  @Override
+  public Boolean visitInputRef(RexInputRef paramRexInputRef) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitLocalRef(RexLocalRef paramRexLocalRef) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitLiteral(RexLiteral paramRexLiteral) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitCall(RexCall paramRexCall) {
+    if(paramRexCall.getOperator() instanceof DrillSqlOperator){
+      return false;
+    }else{
+      for (RexNode operand : paramRexCall.operands) {
+        if (!operand.accept(this)) {
+          return false;
+        }
+      }
+    }
+    return true;
+  }
+
+  public Boolean visitOver(RexOver over) {
+    if (!visitCall(over)) {
+      return false;
+    }
+    ;
+
+    final RexWindow window = over.getWindow();
+    for (RexFieldCollation orderKey : window.orderKeys) {
+      if (!((RexNode) orderKey.left).accept(this)) {
+        return false;
+      }
+    }
+
+    for (RexNode partitionKey : window.partitionKeys) {
+      if (!partitionKey.accept(this)) {
+        return false;
+      }
+    }
+
+    return true;
+
+  }
+
+  @Override
+  public Boolean visitCorrelVariable(RexCorrelVariable paramRexCorrelVariable) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitDynamicParam(RexDynamicParam paramRexDynamicParam) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitRangeRef(RexRangeRef paramRexRangeRef) {
+    return true;
+  }
+
+  @Override
+  public Boolean visitFieldAccess(RexFieldAccess paramRexFieldAccess) {
+    return paramRexFieldAccess.getReferenceExpr().accept(this);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
new file mode 100644
index 0000000..95b03cf
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcGroupScan.java
@@ -0,0 +1,107 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.ScanStats.GroupScanProperty;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("jdbc-scan")
+public class JdbcGroupScan extends AbstractGroupScan {
+
+  private final String sql;
+  private final JdbcStoragePlugin plugin;
+  private final double rows;
+
+  @JsonCreator
+  public JdbcGroupScan(
+      @JsonProperty("sql") String sql,
+      @JsonProperty("config") StoragePluginConfig config,
+      @JsonProperty("rows") double rows,
+      @JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
+    super("");
+    this.sql = sql;
+    this.plugin = (JdbcStoragePlugin) plugins.getPlugin(config);
+    this.rows = rows;
+  }
+
+  JdbcGroupScan(String sql, JdbcStoragePlugin plugin, double rows) {
+    super("");
+    this.sql = sql;
+    this.plugin = plugin;
+    this.rows = rows;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
+  }
+
+  @Override
+  public SubScan getSpecificScan(int minorFragmentId) throws ExecutionSetupException {
+    return new JdbcSubScan(sql, plugin);
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return 1;
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    return new ScanStats(
+        GroupScanProperty.NO_EXACT_ROW_COUNT,
+        (long) Math.max(rows, 1),
+        1,
+        1);
+  }
+
+  public String getSql() {
+    return sql;
+  }
+
+  @Override
+  public String getDigest() {
+    return sql + String.valueOf(plugin.getConfig());
+  }
+
+  public StoragePluginConfig getConfig() {
+    return plugin.getConfig();
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+    return new JdbcGroupScan(sql, plugin, rows);
+  }
+
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcIntermediatePrel.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcIntermediatePrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcIntermediatePrel.java
new file mode 100644
index 0000000..0adb5e0
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcIntermediatePrel.java
@@ -0,0 +1,79 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.rel.RelNode;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.SinglePrel;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.planner.sql.handlers.PrelFinalizable;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+
+/**
+ * Prel used to represent a JDBC Conversion within an expression tree. This Prel will replaced with a full JdbcPrel
+ * before execution can happen.
+ */
+public class JdbcIntermediatePrel extends SinglePrel implements PrelFinalizable {
+
+  public JdbcIntermediatePrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new JdbcIntermediatePrel(getCluster(), traitSet, getInput());
+  }
+
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return copy(getTraitSet(), getInputs());
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public Prel finalizeRel() {
+    return new JdbcPrel(getCluster(), getTraitSet(), this);
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    throw new UnsupportedOperationException("This needs to be finalized before using a PrelVisitor.");
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
new file mode 100644
index 0000000..2433fbd
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcPrel.java
@@ -0,0 +1,120 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.adapter.jdbc.JdbcImplementor;
+import org.apache.calcite.plan.ConventionTraitDef;
+import org.apache.calcite.plan.RelOptCluster;
+import org.apache.calcite.plan.RelTraitSet;
+import org.apache.calcite.plan.volcano.RelSubset;
+import org.apache.calcite.rel.AbstractRelNode;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.RelShuttleImpl;
+import org.apache.calcite.rel.RelWriter;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.visitor.PrelVisitor;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.apache.drill.exec.store.jdbc.JdbcStoragePlugin.DrillJdbcConvention;
+
+/**
+ * Represents a JDBC Plan once the children nodes have been rewritten into SQL.
+ */
+public class JdbcPrel extends AbstractRelNode implements Prel {
+
+  private final String sql;
+  private final double rows;
+  private final DrillJdbcConvention convention;
+
+  public JdbcPrel(RelOptCluster cluster, RelTraitSet traitSet, JdbcIntermediatePrel prel) {
+    super(cluster, traitSet);
+    final RelNode input = prel.getInput();
+    rows = input.getRows();
+    convention = (DrillJdbcConvention) input.getTraitSet().getTrait(ConventionTraitDef.INSTANCE);
+
+    // generate sql for tree.
+    final SqlDialect dialect = convention.getPlugin().getDialect();
+    final JdbcImplementor jdbcImplementor = new JdbcImplementor(
+        dialect,
+        (JavaTypeFactory) getCluster().getTypeFactory());
+    final JdbcImplementor.Result result =
+        jdbcImplementor.visitChild(0, input.accept(new SubsetRemover()));
+    sql = result.asQuery().toSqlString(dialect).getSql();
+    rowType = input.getRowType();
+  }
+
+  private class SubsetRemover extends RelShuttleImpl {
+
+    @Override
+    public RelNode visit(RelNode other) {
+      if (other instanceof RelSubset) {
+        return ((RelSubset) other).getBest().accept(this);
+      } else {
+        return super.visit(other);
+      }
+    }
+
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    return new JdbcGroupScan(sql, convention.getPlugin(), rows);
+  }
+
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw).item("sql", sql);
+  }
+
+  @Override
+  public double getRows() {
+    return rows;
+  }
+
+  @Override
+  public Iterator<Prel> iterator() {
+    return Collections.emptyIterator();
+  }
+
+  @Override
+  public <T, X, E extends Throwable> T accept(PrelVisitor<T, X, E> logicalVisitor, X value) throws E {
+    return logicalVisitor.visitPrel(this, value);
+  }
+
+  @Override
+  public SelectionVectorMode[] getSupportedEncodings() {
+    return SelectionVectorMode.DEFAULT;
+  }
+
+  @Override
+  public SelectionVectorMode getEncoding() {
+    return SelectionVectorMode.NONE;
+  }
+
+  @Override
+  public boolean needsFinalColumnReordering() {
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
new file mode 100755
index 0000000..69c45c2
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcRecordReader.java
@@ -0,0 +1,431 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.math.BigDecimal;
+import java.sql.Connection;
+import java.sql.Date;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.sql.Time;
+import java.sql.Timestamp;
+
+import javax.sql.DataSource;
+
+import org.apache.drill.common.AutoCloseables;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.UserException;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableBitVector;
+import org.apache.drill.exec.vector.NullableDateVector;
+import org.apache.drill.exec.vector.NullableFloat4Vector;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableTimeVector;
+import org.apache.drill.exec.vector.NullableVarBinaryVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.ValueVector;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableMap;
+
+@SuppressWarnings("unchecked")
+class JdbcRecordReader extends AbstractRecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
+      .getLogger(JdbcRecordReader.class);
+
+  private static final ImmutableMap<Integer, MinorType> JDBC_TYPE_MAPPINGS;
+  private final DataSource source;
+  private ResultSet resultSet;
+  private final String storagePluginName;
+  private FragmentContext fragmentContext;
+  private Connection connection;
+  private Statement statement;
+  private final String sql;
+  private ImmutableList<ValueVector> vectors;
+  private ImmutableList<Copier<?>> copiers;
+
+  private OperatorContext operatorContext;
+
+  public JdbcRecordReader(FragmentContext fragmentContext, DataSource source, String sql, String storagePluginName) {
+    this.fragmentContext = fragmentContext;
+    this.source = source;
+    this.sql = sql;
+    this.storagePluginName = storagePluginName;
+  }
+
+  static {
+    JDBC_TYPE_MAPPINGS = (ImmutableMap<Integer, MinorType>) (Object) ImmutableMap.builder()
+        .put(java.sql.Types.DOUBLE, MinorType.FLOAT8)
+        .put(java.sql.Types.FLOAT, MinorType.FLOAT4)
+        .put(java.sql.Types.TINYINT, MinorType.INT)
+        .put(java.sql.Types.SMALLINT, MinorType.INT)
+        .put(java.sql.Types.INTEGER, MinorType.INT)
+        .put(java.sql.Types.BIGINT, MinorType.BIGINT)
+
+        .put(java.sql.Types.CHAR, MinorType.VARCHAR)
+        .put(java.sql.Types.VARCHAR, MinorType.VARCHAR)
+        .put(java.sql.Types.LONGVARCHAR, MinorType.VARCHAR)
+
+        .put(java.sql.Types.NCHAR, MinorType.VARCHAR)
+        .put(java.sql.Types.NVARCHAR, MinorType.VARCHAR)
+        .put(java.sql.Types.LONGNVARCHAR, MinorType.VARCHAR)
+
+        .put(java.sql.Types.VARBINARY, MinorType.VARBINARY)
+        .put(java.sql.Types.LONGVARBINARY, MinorType.VARBINARY)
+
+        .put(java.sql.Types.NUMERIC, MinorType.FLOAT8)
+        .put(java.sql.Types.DECIMAL, MinorType.FLOAT8)
+        .put(java.sql.Types.REAL, MinorType.FLOAT8)
+
+        .put(java.sql.Types.DATE, MinorType.DATE)
+        .put(java.sql.Types.TIME, MinorType.TIME)
+        .put(java.sql.Types.TIMESTAMP, MinorType.TIMESTAMP)
+
+        .put(java.sql.Types.BOOLEAN, MinorType.BIT)
+
+        .build();
+  }
+
+  private Copier<?> getCopier(int jdbcType, int offset, ResultSet result, ValueVector v) {
+
+    if (v instanceof NullableBigIntVector) {
+      return new BigIntCopier(offset, result, (NullableBigIntVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableFloat4Vector) {
+      return new Float4Copier(offset, result, (NullableFloat4Vector.Mutator) v.getMutator());
+    } else if (v instanceof NullableFloat8Vector) {
+      return new Float8Copier(offset, result, (NullableFloat8Vector.Mutator) v.getMutator());
+    } else if (v instanceof NullableIntVector) {
+      return new IntCopier(offset, result, (NullableIntVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableVarCharVector) {
+      return new VarCharCopier(offset, result, (NullableVarCharVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableVarBinaryVector) {
+      return new VarBinaryCopier(offset, result, (NullableVarBinaryVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableDateVector) {
+      return new DateCopier(offset, result, (NullableDateVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableTimeVector) {
+      return new TimeCopier(offset, result, (NullableTimeVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableTimeStampVector) {
+      return new TimeStampCopier(offset, result, (NullableTimeStampVector.Mutator) v.getMutator());
+    } else if (v instanceof NullableBitVector) {
+      return new BitCopier(offset, result, (NullableBitVector.Mutator) v.getMutator());
+    }
+
+    throw new IllegalArgumentException("Unknown how to handle vector.");
+  }
+
+  @Override
+  public void setup(OperatorContext operatorContext, OutputMutator output) throws ExecutionSetupException {
+    try {
+
+      this.operatorContext = operatorContext;
+      connection = source.getConnection();
+      statement = connection.createStatement();
+      resultSet = statement.executeQuery(sql);
+
+      final ResultSetMetaData meta = resultSet.getMetaData();
+      final int columns = meta.getColumnCount();
+      ImmutableList.Builder<ValueVector> vectorBuilder = ImmutableList.builder();
+      ImmutableList.Builder<Copier<?>> copierBuilder = ImmutableList.builder();
+
+      for (int i = 1; i <= columns; i++) {
+        final String name = meta.getColumnLabel(i);
+        final int jdbcType = meta.getColumnType(i);
+        final int width = meta.getPrecision(i);
+        final int scale = meta.getScale(i);
+        MinorType minorType = JDBC_TYPE_MAPPINGS.get(jdbcType);
+        if (minorType == null) {
+          throw UserException.dataReadError()
+              .message("The JDBC storage plugin failed while trying to execute a query. "
+                  + "The JDBC data type %d is not currently supported.", jdbcType)
+
+              .addContext("sql", sql)
+              .addContext("plugin", storagePluginName)
+              .build(logger);
+        }
+
+        final MajorType type = Types.optional(minorType);
+        final MaterializedField field = MaterializedField.create(name, type);
+        final Class<? extends ValueVector> clazz = (Class<? extends ValueVector>) TypeHelper.getValueVectorClass(
+            minorType, type.getMode());
+        ValueVector vector = output.addField(field, clazz);
+        vectorBuilder.add(vector);
+        copierBuilder.add(getCopier(jdbcType, i, resultSet, vector));
+
+      }
+
+      vectors = vectorBuilder.build();
+      copiers = copierBuilder.build();
+
+    } catch (SQLException | SchemaChangeException e) {
+      throw UserException.dataReadError(e)
+          .message("The JDBC storage plugin failed while trying setup the SQL query. ")
+          .addContext("sql", sql)
+          .addContext("plugin", storagePluginName)
+          .build(logger);
+    }
+  }
+
+
+  @Override
+  public int next() {
+    int counter = 0;
+    Boolean b = true;
+    try {
+      while (counter < 4095 && b == true) { // loop at 4095 since nullables use one more than record count and we
+                                            // allocate on powers of two.
+        b = resultSet.next();
+        if(b == false) {
+            break;
+        }
+        for (Copier<?> c : copiers) {
+          c.copy(counter);
+        }
+        counter++;
+      }
+    } catch (SQLException e) {
+      throw UserException
+          .dataReadError(e)
+          .message("Failure while attempting to read from database.")
+          .addContext("sql", sql)
+          .addContext("plugin", storagePluginName)
+          .build(logger);
+    }
+
+    for (ValueVector vv : vectors) {
+      vv.getMutator().setValueCount(counter > 0 ? counter : 0);
+    }
+
+    return counter>0 ? counter : 0;
+  }
+
+  @Override
+  public void cleanup() {
+    AutoCloseables.close(resultSet, logger);
+    AutoCloseables.close(statement, logger);
+    AutoCloseables.close(connection, logger);
+  }
+
+  private abstract class Copier<T extends ValueVector.Mutator> {
+    protected final int columnIndex;
+    protected final ResultSet result;
+    protected final T mutator;
+
+    public Copier(int columnIndex, ResultSet result, T mutator) {
+      super();
+      this.columnIndex = columnIndex;
+      this.result = result;
+      this.mutator = mutator;
+    }
+
+    abstract void copy(int index) throws SQLException;
+  }
+
+  private class IntCopier extends Copier<NullableIntVector.Mutator> {
+    public IntCopier(int offset, ResultSet set, NullableIntVector.Mutator mutator) {
+      super(offset, set, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      mutator.setSafe(index, result.getInt(columnIndex));
+      if (result.wasNull()) {
+        mutator.setNull(index);
+      }
+    }
+  }
+
+  private class BigIntCopier extends Copier<NullableBigIntVector.Mutator> {
+    public BigIntCopier(int offset, ResultSet set, NullableBigIntVector.Mutator mutator) {
+      super(offset, set, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      mutator.setSafe(index, result.getLong(columnIndex));
+      if (result.wasNull()) {
+        mutator.setNull(index);
+      }
+    }
+
+  }
+
+  private class Float4Copier extends Copier<NullableFloat4Vector.Mutator> {
+
+    public Float4Copier(int columnIndex, ResultSet result, NullableFloat4Vector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      mutator.setSafe(index, result.getFloat(columnIndex));
+      if (result.wasNull()) {
+        mutator.setNull(index);
+      }
+    }
+
+  }
+
+
+  private class Float8Copier extends Copier<NullableFloat8Vector.Mutator> {
+
+    public Float8Copier(int columnIndex, ResultSet result, NullableFloat8Vector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      mutator.setSafe(index, result.getDouble(columnIndex));
+      if (result.wasNull()) {
+        mutator.setNull(index);
+      }
+
+    }
+
+  }
+
+  private class DecimalCopier extends Copier<NullableFloat8Vector.Mutator> {
+
+    public DecimalCopier(int columnIndex, ResultSet result, NullableFloat8Vector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      BigDecimal decimal = result.getBigDecimal(columnIndex);
+      if (decimal != null) {
+        mutator.setSafe(index, decimal.doubleValue());
+      }
+    }
+
+  }
+
+  private class VarCharCopier extends Copier<NullableVarCharVector.Mutator> {
+
+    public VarCharCopier(int columnIndex, ResultSet result, NullableVarCharVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      String val = resultSet.getString(columnIndex);
+      if (val != null) {
+        byte[] record = val.getBytes(Charsets.UTF_8);
+        mutator.setSafe(index, record, 0, record.length);
+      }
+    }
+
+  }
+
+  private class VarBinaryCopier extends Copier<NullableVarBinaryVector.Mutator> {
+
+    public VarBinaryCopier(int columnIndex, ResultSet result, NullableVarBinaryVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      byte[] record = result.getBytes(columnIndex);
+      if (record != null) {
+        mutator.setSafe(index, record, 0, record.length);
+      }
+    }
+
+  }
+
+  private class DateCopier extends Copier<NullableDateVector.Mutator> {
+
+    public DateCopier(int columnIndex, ResultSet result, NullableDateVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      Date date = result.getDate(columnIndex);
+      if (date != null) {
+        mutator.setSafe(index, date.getTime());
+      }
+    }
+
+  }
+
+  private class TimeCopier extends Copier<NullableTimeVector.Mutator> {
+
+    public TimeCopier(int columnIndex, ResultSet result, NullableTimeVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      Time time = result.getTime(columnIndex);
+      if (time != null) {
+        mutator.setSafe(index, (int) time.getTime());
+      }
+
+    }
+
+  }
+
+  private class TimeStampCopier extends Copier<NullableTimeStampVector.Mutator> {
+
+    public TimeStampCopier(int columnIndex, ResultSet result, NullableTimeStampVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      Timestamp stamp = result.getTimestamp(columnIndex);
+      if (stamp != null) {
+        mutator.setSafe(index, stamp.getTime());
+      }
+
+    }
+
+  }
+
+  private class BitCopier extends Copier<NullableBitVector.Mutator> {
+
+    public BitCopier(int columnIndex, ResultSet result, NullableBitVector.Mutator mutator) {
+      super(columnIndex, result, mutator);
+    }
+
+    @Override
+    void copy(int index) throws SQLException {
+      mutator.setSafe(index, result.getBoolean(columnIndex) ? 1 : 0);
+      if (result.wasNull()) {
+        mutator.setNull(index);
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
new file mode 100755
index 0000000..5a921d4
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStorageConfig.java
@@ -0,0 +1,120 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName(JdbcStorageConfig.NAME)
+public class JdbcStorageConfig extends StoragePluginConfig {
+
+  public static final String NAME = "jdbc";
+
+  private final String driver;
+  private final String url;
+  private final String username;
+  private final String password;
+
+  @JsonCreator
+  public JdbcStorageConfig(
+      @JsonProperty("driver") String driver,
+      @JsonProperty("url") String url,
+      @JsonProperty("username") String username,
+      @JsonProperty("password") String password) {
+    super();
+    this.driver = driver;
+    this.url = url;
+    this.username = username;
+    this.password = password;
+  }
+
+  public String getDriver() {
+    return driver;
+  }
+
+  public String getUrl() {
+    return url;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public String getPassword() {
+    return password;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((driver == null) ? 0 : driver.hashCode());
+    result = prime * result + ((password == null) ? 0 : password.hashCode());
+    result = prime * result + ((url == null) ? 0 : url.hashCode());
+    result = prime * result + ((username == null) ? 0 : username.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    JdbcStorageConfig other = (JdbcStorageConfig) obj;
+    if (driver == null) {
+      if (other.driver != null) {
+        return false;
+      }
+    } else if (!driver.equals(other.driver)) {
+      return false;
+    }
+    if (password == null) {
+      if (other.password != null) {
+        return false;
+      }
+    } else if (!password.equals(other.password)) {
+      return false;
+    }
+    if (url == null) {
+      if (other.url != null) {
+        return false;
+      }
+    } else if (!url.equals(other.url)) {
+      return false;
+    }
+    if (username == null) {
+      if (other.username != null) {
+        return false;
+      }
+    } else if (!username.equals(other.username)) {
+      return false;
+    }
+    return true;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
new file mode 100755
index 0000000..f27f6f1
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcStoragePlugin.java
@@ -0,0 +1,400 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import javax.sql.DataSource;
+
+import org.apache.calcite.adapter.jdbc.JdbcConvention;
+import org.apache.calcite.adapter.jdbc.JdbcRules;
+import org.apache.calcite.adapter.jdbc.JdbcRules.JdbcJoin;
+import org.apache.calcite.adapter.jdbc.JdbcSchema;
+import org.apache.calcite.linq4j.tree.ConstantUntypedNull;
+import org.apache.calcite.plan.RelOptPlanner;
+import org.apache.calcite.plan.RelOptRule;
+import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.rel.convert.ConverterRule;
+import org.apache.calcite.rel.rules.FilterSetOpTransposeRule;
+import org.apache.calcite.rel.rules.ProjectRemoveRule;
+import org.apache.calcite.rex.RexCall;
+import org.apache.calcite.rex.RexInputRef;
+import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.sql.SqlDialect;
+import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.OptimizerRulesContext;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractSchema;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.jdbc.DrillJdbcRuleBase.DrillJdbcFilterRule;
+import org.apache.drill.exec.store.jdbc.DrillJdbcRuleBase.DrillJdbcProjectRule;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+
+public class JdbcStoragePlugin extends AbstractStoragePlugin {
+  private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(JdbcStoragePlugin.class);
+
+  // Rules from Calcite's JdbcRules class that we want to avoid using.
+  private static String[] RULES_TO_AVOID = {
+      "JdbcToEnumerableConverterRule", "JdbcFilterRule", "JdbcProjectRule"
+  };
+
+
+  private final JdbcStorageConfig config;
+  private final DrillbitContext context;
+  private final DataSource source;
+  private final String name;
+  private final SqlDialect dialect;
+  private final DrillJdbcConvention convention;
+
+
+  public JdbcStoragePlugin(JdbcStorageConfig config, DrillbitContext context, String name) {
+    this.context = context;
+    this.config = config;
+    this.name = name;
+    BasicDataSource source = new BasicDataSource();
+    source.setDriverClassName(config.getDriver());
+    source.setUrl(config.getUrl());
+
+    if (config.getUsername() != null) {
+      source.setUsername(config.getUsername());
+    }
+
+    if (config.getPassword() != null) {
+      source.setPassword(config.getPassword());
+    }
+
+    this.source = source;
+    this.dialect = JdbcSchema.createDialect(source);
+    this.convention = new DrillJdbcConvention(dialect, name);
+  }
+
+
+  class DrillJdbcConvention extends JdbcConvention {
+
+    private final ImmutableSet<RelOptRule> rules;
+
+    public DrillJdbcConvention(SqlDialect dialect, String name) {
+      super(dialect, ConstantUntypedNull.INSTANCE, name);
+
+
+      // build rules for this convention.
+      ImmutableSet.Builder<RelOptRule> builder = ImmutableSet.builder();
+
+      builder.add(JDBC_PRULE_INSTANCE);
+      builder.add(new JdbcDrelConverterRule(this));
+      builder.add(new DrillJdbcProjectRule(this));
+      builder.add(new DrillJdbcFilterRule(this));
+
+      outside: for (RelOptRule rule : JdbcRules.rules(this)) {
+        final String description = rule.toString();
+
+        // we want to black list some rules but the parent Calcite package is all or none.
+        // Therefore, we remove rules with names we don't like.
+        for(String black : RULES_TO_AVOID){
+          if(description.equals(black)){
+            continue outside;
+          }
+
+        }
+
+        builder.add(rule);
+      }
+
+      builder.add(FilterSetOpTransposeRule.INSTANCE);
+      builder.add(ProjectRemoveRule.INSTANCE);
+
+      rules = builder.build();
+    }
+
+    @Override
+    public void register(RelOptPlanner planner) {
+      for (RelOptRule rule : rules) {
+        planner.addRule(rule);
+      }
+    }
+
+    public Set<RelOptRule> getRules() {
+      return rules;
+    }
+
+    public JdbcStoragePlugin getPlugin() {
+      return JdbcStoragePlugin.this;
+    }
+  }
+
+  /**
+   * Returns whether a condition is supported by {@link JdbcJoin}.
+   *
+   * <p>Corresponds to the capabilities of
+   * {@link JdbcJoin#convertConditionToSqlNode}.
+   *
+   * @param node Condition
+   * @return Whether condition is supported
+   */
+  private static boolean canJoinOnCondition(RexNode node) {
+    final List<RexNode> operands;
+    switch (node.getKind()) {
+    case AND:
+    case OR:
+      operands = ((RexCall) node).getOperands();
+      for (RexNode operand : operands) {
+        if (!canJoinOnCondition(operand)) {
+          return false;
+        }
+      }
+      return true;
+
+    case EQUALS:
+    case IS_NOT_DISTINCT_FROM:
+    case NOT_EQUALS:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+      operands = ((RexCall) node).getOperands();
+      if ((operands.get(0) instanceof RexInputRef)
+          && (operands.get(1) instanceof RexInputRef)) {
+        return true;
+      }
+      // fall through
+
+    default:
+      return false;
+    }
+  }
+
+
+  private static final JdbcPrule JDBC_PRULE_INSTANCE = new JdbcPrule();
+
+  private static class JdbcPrule extends ConverterRule {
+
+    private JdbcPrule() {
+      super(JdbcDrel.class, DrillRel.DRILL_LOGICAL, Prel.DRILL_PHYSICAL, "JDBC_PREL_Converter");
+    }
+
+    @Override
+    public RelNode convert(RelNode in) {
+
+      return new JdbcIntermediatePrel(
+          in.getCluster(),
+          in.getTraitSet().replace(getOutTrait()),
+          in.getInput(0));
+    }
+
+  }
+
+  private class JdbcDrelConverterRule extends ConverterRule {
+
+    public JdbcDrelConverterRule(DrillJdbcConvention in) {
+      super(RelNode.class, in, DrillRel.DRILL_LOGICAL, "JDBC_DREL_Converter" + in.getName());
+    }
+
+    @Override
+    public RelNode convert(RelNode in) {
+      return new JdbcDrel(in.getCluster(), in.getTraitSet().replace(DrillRel.DRILL_LOGICAL),
+          convert(in, in.getTraitSet().replace(this.getInTrait())));
+    }
+
+  }
+
+  private class CapitalizingJdbcSchema extends AbstractSchema {
+
+    private final JdbcSchema inner;
+
+    public CapitalizingJdbcSchema(List<String> parentSchemaPath, String name, DataSource dataSource,
+        SqlDialect dialect, JdbcConvention convention, String catalog, String schema) {
+      super(parentSchemaPath, name);
+      inner = new JdbcSchema(dataSource, dialect, convention, catalog, schema);
+    }
+
+    @Override
+    public String getTypeName() {
+      return JdbcStorageConfig.NAME;
+    }
+
+    @Override
+    public Collection<Function> getFunctions(String name) {
+      return inner.getFunctions(name);
+    }
+
+    @Override
+    public Set<String> getFunctionNames() {
+      return inner.getFunctionNames();
+    }
+
+    @Override
+    public Schema getSubSchema(String name) {
+      return inner.getSubSchema(name);
+    }
+
+    @Override
+    public Set<String> getSubSchemaNames() {
+      return inner.getSubSchemaNames();
+    }
+
+    @Override
+    public Set<String> getTableNames() {
+      return inner.getTableNames();
+    }
+
+    @Override
+    public Table getTable(String name) {
+      Table table = inner.getTable(name);
+      if (table != null) {
+        return table;
+      }
+      return inner.getTable(name.toUpperCase());
+
+    }
+
+  }
+
+  private class JdbcCatalogSchema extends AbstractSchema {
+
+    private final Map<String, CapitalizingJdbcSchema> schemaMap = Maps.newHashMap();
+    private final CapitalizingJdbcSchema defaultSchema;
+
+    public JdbcCatalogSchema(String name) {
+      super(ImmutableList.<String> of(), name);
+
+      try (Connection con = source.getConnection(); ResultSet set = con.getMetaData().getCatalogs()) {
+        while (set.next()) {
+          final String catalogName = set.getString(1);
+          CapitalizingJdbcSchema schema = new CapitalizingJdbcSchema(getSchemaPath(), catalogName, source, dialect,
+              convention, catalogName, null);
+          schemaMap.put(catalogName, schema);
+        }
+      } catch (SQLException e) {
+        logger.warn("Failure while attempting to load JDBC schema.", e);
+      }
+
+      // unable to read general catalog
+      if (schemaMap.isEmpty()) {
+        schemaMap.put("default", new CapitalizingJdbcSchema(ImmutableList.<String> of(), name, source, dialect,
+            convention,
+            null, null));
+      }
+
+      defaultSchema = schemaMap.values().iterator().next();
+
+    }
+
+    @Override
+    public String getTypeName() {
+      return JdbcStorageConfig.NAME;
+    }
+
+    @Override
+    public Schema getDefaultSchema() {
+      return defaultSchema;
+    }
+
+    @Override
+    public Schema getSubSchema(String name) {
+      return schemaMap.get(name);
+    }
+
+    @Override
+    public Set<String> getSubSchemaNames() {
+      return schemaMap.keySet();
+    }
+
+    @Override
+    public Table getTable(String name) {
+      Schema schema = getDefaultSchema();
+      if (schema != null) {
+        Table t = schema.getTable(name);
+        if (t != null) {
+          return t;
+        }
+        return schema.getTable(name.toUpperCase());
+      } else {
+        return null;
+      }
+    }
+
+    @Override
+    public Set<String> getTableNames() {
+      return defaultSchema.getTableNames();
+    }
+
+  }
+
+  @Override
+  public void registerSchemas(SchemaConfig config, SchemaPlus parent) {
+    JdbcCatalogSchema schema = new JdbcCatalogSchema(name);
+    parent.add(name, schema);
+  }
+
+  @Override
+  public JdbcStorageConfig getConfig() {
+    return config;
+  }
+
+  public DrillbitContext getContext() {
+    return this.context;
+  }
+
+  public String getName() {
+    return this.name;
+  }
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+
+  public DataSource getSource() {
+    return source;
+  }
+
+  public SqlDialect getDialect() {
+    return dialect;
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName, JSONOptions selection, List<SchemaPath> columns)
+      throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Set<RelOptRule> getOptimizerRules(OptimizerRulesContext context) {
+    return convention.getRules();
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
new file mode 100755
index 0000000..fcafd4c
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/java/org/apache/drill/exec/store/jdbc/JdbcSubScan.java
@@ -0,0 +1,71 @@
+/**
+ * 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.drill.exec.store.jdbc;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+
+import com.fasterxml.jackson.annotation.JacksonInject;
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+
+@JsonTypeName("jdbc-sub-scan")
+public class JdbcSubScan extends AbstractSubScan {
+
+  private final String sql;
+  private final JdbcStoragePlugin plugin;
+
+  @JsonCreator
+  public JdbcSubScan(
+      @JsonProperty("sql") String sql,
+      @JsonProperty("config") StoragePluginConfig config,
+      @JacksonInject StoragePluginRegistry plugins) throws ExecutionSetupException {
+    super("");
+    this.sql = sql;
+    this.plugin = (JdbcStoragePlugin) plugins.getPlugin(config);
+  }
+
+  JdbcSubScan(String sql, JdbcStoragePlugin plugin) {
+    super("");
+    this.sql = sql;
+    this.plugin = plugin;
+  }
+
+  @Override
+  public int getOperatorType() {
+    return -1;
+  }
+
+  public String getSql() {
+    return sql;
+  }
+
+  public StoragePluginConfig getConfig() {
+    return plugin.getConfig();
+  }
+
+  @JsonIgnore
+  public JdbcStoragePlugin getPlugin() {
+    return plugin;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/resources/bootstrap-storage-plugins.json b/contrib/storage-jdbc/src/main/resources/bootstrap-storage-plugins.json
new file mode 100755
index 0000000..7d88052
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/resources/bootstrap-storage-plugins.json
@@ -0,0 +1,10 @@
+{
+  "storage":{
+    "jdbc" : {
+      type:"jdbc",
+      enabled: false,
+      driver:"org.apache.derby.jdbc.ClientDriver",
+      url:"jdbc:derby://localhost:20000/memory:testDB;"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/main/resources/drill-module.conf
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/main/resources/drill-module.conf b/contrib/storage-jdbc/src/main/resources/drill-module.conf
new file mode 100755
index 0000000..721a599
--- /dev/null
+++ b/contrib/storage-jdbc/src/main/resources/drill-module.conf
@@ -0,0 +1,18 @@
+// 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.
+//
+//  This file tells Drill to consider this module when class path scanning.
+//  This file can also include any supplementary configuration information.
+//  This file is in HOCON format, see https://github.com/typesafehub/config/blob/master/HOCON.md for more information.

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPlugin.java b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPlugin.java
new file mode 100644
index 0000000..1f15068
--- /dev/null
+++ b/contrib/storage-jdbc/src/test/java/org/apache/drill/exec/store/jdbc/TestJdbcPlugin.java
@@ -0,0 +1,181 @@
+package org.apache.drill.exec.store.jdbc;
+
+import static org.junit.Assert.assertEquals;
+
+import java.net.InetAddress;
+import java.sql.Connection;
+
+import org.apache.commons.dbcp.BasicDataSource;
+import org.apache.derby.drda.NetworkServerControl;
+import org.apache.drill.BaseTestQuery;
+import org.apache.drill.PlanTestBase;
+import org.apache.drill.exec.proto.UserBitShared.QueryType;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+/**
+ * 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.
+ */
+public class TestJdbcPlugin extends PlanTestBase {
+
+  static NetworkServerControl server;
+
+  @BeforeClass
+  public static void setupDefaultTestCluster() throws Exception {
+    System.setProperty("derby.drda.startNetworkServer", "true");
+    server = new NetworkServerControl(InetAddress.getByName("localhost"),
+        20000,
+        "admin",
+        "admin");
+    java.io.PrintWriter consoleWriter = new java.io.PrintWriter(System.out, true);
+    server.start(consoleWriter);
+
+    BasicDataSource source = new BasicDataSource();
+    source.setUrl("jdbc:derby://localhost:20000/memory:testDB;create=true");
+    source.setDriverClassName("org.apache.derby.jdbc.ClientDriver");
+
+    final String insertValues1 = "INSERT INTO person VALUES (1, 'Smith', null, '{number:\"123 Main\"}','mtrx', "
+        + "'xy', 333.333, 444.444, 555.00, TIME('15:09:02'), DATE('1994-02-23'), TIMESTAMP('1962-09-23 03:23:34.234'),"
+        + " 666.66, 1, -1, false)";
+    final String insertValues2 = "INSERT INTO person (PersonId) VALUES (null)";
+    try (Connection c = source.getConnection()) {
+      c.createStatement().execute("CREATE TABLE person\n" +
+          "(\n" +
+          "PersonID int,\n" +
+          "LastName varchar(255),\n" +
+          "FirstName varchar(255),\n" +
+          "Address varchar(255),\n" +
+          "City varchar(255),\n" +
+          "Code char(2),\n" +
+          "dbl double,\n" +
+          "flt float,\n" +
+          "rel real,\n" +
+          "tm time,\n" +
+          "dt date,\n" +
+          "tms timestamp,\n" +
+          "num numeric(10,2), \n" +
+          "sm smallint,\n" +
+          "bi bigint,\n" +
+          "bool boolean\n" +
+
+          ")");
+
+      c.createStatement().execute(insertValues1);
+      c.createStatement().execute(insertValues2);
+      c.createStatement().execute(insertValues1);
+    }
+
+    BaseTestQuery.setupDefaultTestCluster();
+  }
+
+  @AfterClass
+  public static void shutdownDb() throws Exception {
+    server.shutdown();
+  }
+
+  @Test
+  public void validateResult() throws Exception {
+    // we'll test data except for date, time and timestamps. Derby mangles these due to improper timezone support.
+    testBuilder()
+        .sqlQuery(
+            "select PERSONID, LASTNAME, FIRSTNAME, ADDRESS, CITY, CODE, DBL, FLT, REL, NUM, SM, BI, BOOL from testdb.`default`.PERSON")
+        .ordered()
+        .baselineColumns("PERSONID", "LASTNAME", "FIRSTNAME", "ADDRESS", "CITY", "CODE", "DBL", "FLT", "REL",
+            "NUM", "SM", "BI", "BOOL")
+        .baselineValues(1, "Smith", null, "{number:\"123 Main\"}", "mtrx", "xy", 333.333, 444.444, 555.00,
+            666.66, 1, -1l, false)
+        .baselineValues(null, null, null, null, null, null, null, null, null, null, null, null, null)
+        .baselineValues(1, "Smith", null, "{number:\"123 Main\"}", "mtrx", "xy", 333.333, 444.444, 555.00,
+            666.66, 1, -1l, false)
+        .build().run();
+  }
+
+  @Test
+  public void queryDefaultSchema() throws Exception {
+    testNoResult("select * from testdb.PERSON");
+  }
+
+  @Test
+  public void queryDifferentCase() throws Exception {
+    testNoResult("select * from testdb.person");
+  }
+
+  @Test
+  public void pushdownJoin() throws Exception {
+    testNoResult("use testdb");
+    String query = "select x.PersonId from (select PersonId from person)x "
+        + "join (select PersonId from person)y on x.PersonId = y.PersonId ";
+    testPlanMatchingPatterns(query, new String[] {}, new String[] { "Join" });
+
+  }
+
+  @Test
+  public void pushdownJoinAndFilterPushDown() throws Exception {
+    final String query = "select * from \n" +
+        "testdb.`default`.PERSON e\n" +
+        "INNER JOIN \n" +
+        "testdb.`default`.PERSON s\n" +
+        "ON e.FirstName = s.FirstName\n" +
+        "WHERE e.LastName > 'hello'";
+
+    testPlanMatchingPatterns(query, new String[] {}, new String[] { "Join", "Filter" });
+  }
+
+  @Test
+  public void pushdownAggregation() throws Exception {
+    final String query = "select count(*) from \n" +
+        "testdb.`default`.PERSON";
+
+    testPlanMatchingPatterns(query, new String[] {}, new String[] { "Aggregate" });
+  }
+
+  @Test
+  public void pushdownDoubleJoinAndFilter() throws Exception {
+    final String query = "select * from \n" +
+        "testdb.`default`.PERSON e\n" +
+        "INNER JOIN \n" +
+        "testdb.`default`.PERSON s\n" +
+        "ON e.PersonId = s.PersonId\n" +
+        "INNER JOIN \n" +
+        "testdb.`default`.PERSON ed\n" +
+        "ON e.PersonId = ed.PersonId\n" +
+        "WHERE s.FirstName > 'abc' and ed.FirstName > 'efg'";
+    testPlanMatchingPatterns(query, new String[] {}, new String[] { "Join", "Filter" });
+  }
+
+  @Test
+  public void showTablesDefaultSchema() throws Exception {
+    testNoResult("use testdb");
+    assertEquals(1, testRunAndPrint(QueryType.SQL, "show tables like 'PERSON'"));
+  }
+
+  @Test
+  public void describe() throws Exception {
+    testNoResult("use testdb");
+    assertEquals(16, testRunAndPrint(QueryType.SQL, "describe PERSON"));
+  }
+
+  @Test
+  public void ensureDrillFunctionsAreNotPushedDown() throws Exception {
+    // This should verify that we're not trying to push CONVERT_FROM into the JDBC storage plugin. If were pushing
+    // this function down, the SQL query would fail.
+    testNoResult("select CONVERT_FROM(Address, 'JSON') from testdb.person where PersonId = 1");
+  }
+
+  @Test
+  public void pushdownFilter() throws Exception {
+    testNoResult("use testdb");
+    String query = "select * from person where PersonId = 1";
+    testPlanMatchingPatterns(query, new String[] {}, new String[] { "Filter" });
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
new file mode 100755
index 0000000..200ab93
--- /dev/null
+++ b/contrib/storage-jdbc/src/test/resources/bootstrap-storage-plugins.json
@@ -0,0 +1,10 @@
+{
+  "storage":{
+    testdb : {
+      type:"jdbc",
+      enabled: true,
+      driver:"org.apache.derby.jdbc.ClientDriver",
+      url:"jdbc:derby://localhost:20000/memory:testDB;"
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-jdbc/src/test/resources/logback.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-jdbc/src/test/resources/logback.xml b/contrib/storage-jdbc/src/test/resources/logback.xml
new file mode 100644
index 0000000..5facafe
--- /dev/null
+++ b/contrib/storage-jdbc/src/test/resources/logback.xml
@@ -0,0 +1,48 @@
+<?xml version="1.0" encoding="UTF-8" ?>
+<!--
+ 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.
+-->
+<configuration>
+
+  <appender name="SOCKET" class="de.huxhorn.lilith.logback.appender.ClassicMultiplexSocketAppender">
+    <Compressing>true</Compressing>
+    <ReconnectionDelay>10000</ReconnectionDelay>
+    <IncludeCallerData>true</IncludeCallerData>
+    <RemoteHosts>${LILITH_HOSTNAME:-localhost}</RemoteHosts>
+  </appender>
+
+ <appender name="STDOUT" class="ch.qos.logback.core.ConsoleAppender">
+    <!-- encoders are assigned the type
+         ch.qos.logback.classic.encoder.PatternLayoutEncoder by default -->
+    <encoder>
+      <pattern>%d{HH:mm:ss.SSS} [%thread] %-5level %logger{36} - %msg%n</pattern>
+    </encoder>
+    <level value="warn" />
+  </appender>
+
+  <logger name="org.apache.drill" additivity="false">
+    <level value="debug" />
+    <appender-ref ref="SOCKET" />
+<!--     <appender-ref ref="STDOUT" /> -->
+  </logger>
+
+  <root>
+    <level value="debug" />
+    <appender-ref ref="SOCKET" />
+<!--     <appender-ref ref="STDOUT" /> -->
+  </root>
+
+</configuration>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/pom.xml b/contrib/storage-mpjdbc/pom.xml
deleted file mode 100755
index 5e9afca..0000000
--- a/contrib/storage-mpjdbc/pom.xml
+++ /dev/null
@@ -1,79 +0,0 @@
-<?xml version="1.0"?>
-<!--
- Licensed to the Apache Software Foundation (ASF) under one or more
- contributor license agreements.  See the NOTICE file distributed with
- this work for additional information regarding copyright ownership.
- The ASF licenses this file to You under the Apache License, Version 2.0
- (the "License"); you may not use this file except in compliance with
- the License.  You may obtain a copy of the License at
-
-     http://www.apache.org/licenses/LICENSE-2.0
-
- Unless required by applicable law or agreed to in writing, software
- distributed under the License is distributed on an "AS IS" BASIS,
- WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- See the License for the specific language governing permissions and
- limitations under the License.
--->
-<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
-    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
-  <modelVersion>4.0.0</modelVersion>
-  <parent>
-    <artifactId>drill-contrib-parent</artifactId>
-    <groupId>org.apache.drill.contrib</groupId>
-    <version>0.9.0-SNAPSHOT</version>
-  </parent>
-
-  <artifactId>drill-mpjdbc-storage</artifactId>
-
-  <name>contrib/mpjdbc-storage-plugin</name>
-
-  <dependencies>
-    <dependency>
-      <groupId>org.apache.drill.exec</groupId>
-      <artifactId>drill-java-exec</artifactId>
-      <version>${project.version}</version>
-    </dependency>
-    
-
-    <!-- Test dependencie -->
-    <dependency>
-      <groupId>org.apache.drill.exec</groupId>
-      <artifactId>drill-java-exec</artifactId>
-      <classifier>tests</classifier>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>org.apache.drill</groupId>
-      <artifactId>drill-common</artifactId>
-      <classifier>tests</classifier>
-      <version>${project.version}</version>
-      <scope>test</scope>
-    </dependency>
-    <dependency>
-      <groupId>com.yammer.metrics</groupId>
-      <artifactId>metrics-core</artifactId>
-      <version>2.1.1</version>
-      <scope>test</scope>
-    </dependency>
-  </dependencies>
-
-  <build>
-    <plugins>
-      <plugin>
-        <groupId>org.apache.maven.plugins</groupId>
-        <artifactId>maven-surefire-plugin</artifactId>
-        <configuration>
-          <systemProperties>
-            <property>
-              <name>logback.log.dir</name>
-              <value>${project.build.directory}/surefire-reports</value>
-            </property>
-          </systemProperties>
-        </configuration>
-      </plugin>
-    </plugins>
-  </build>
-  
-</project>

http://git-wip-us.apache.org/repos/asf/drill/blob/e12cd470/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
deleted file mode 100644
index af08b2e..0000000
--- a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.drill.exec.store.mpjdbc;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.SchemaPath;
-import org.apache.drill.exec.ops.FragmentContext;
-import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.physical.impl.BatchCreator;
-import org.apache.drill.exec.physical.impl.ScanBatch;
-//import org.apache.drill.exec.record.CloseableRecordBatch;
-import org.apache.drill.exec.record.RecordBatch;
-import org.apache.drill.exec.store.RecordReader;
-import org.apache.drill.exec.store.mpjdbc.MPJdbcSchemaSubScan;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-
-public class MPJdbcBatchCreator implements BatchCreator<MPJdbcSubScan> {
-  @Override
-  public RecordBatch getBatch(FragmentContext context, MPJdbcSubScan config,
-      List<RecordBatch> children) throws ExecutionSetupException {
-    Preconditions.checkArgument(children.isEmpty());
-    List<RecordReader> readers = Lists.newArrayList();
-    List<SchemaPath> columns = null;
-    try {
-      if ((columns = config.getColumns()) == null) {
-        columns = GroupScan.ALL_COLUMNS;
-      }
-      readers.add(new MPJdbcRecordReader(context,config));
-    } catch (Exception e1) {
-      throw new ExecutionSetupException(e1);
-    }
-    return new ScanBatch(config, context, readers.iterator());
-  }
-}


[09/15] drill git commit: DRILL-3720: Avro Record Reader should process Avro files by per block basis

Posted by ja...@apache.org.
DRILL-3720: Avro Record Reader should process Avro files by per block basis


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

Branch: refs/heads/master
Commit: 8f4ca6ee851d1f504bd8f34a6329580a177d361d
Parents: 2214894
Author: Kamesh <ka...@gmail.com>
Authored: Fri Aug 28 15:20:44 2015 +0530
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 21:58:33 2015 -0700

----------------------------------------------------------------------
 .../drill/exec/store/avro/AvroFormatPlugin.java       |  4 ++--
 .../drill/exec/store/avro/AvroRecordReader.java       | 14 ++++++++++++--
 2 files changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8f4ca6ee/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
index 30c45fa..5a73a9b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroFormatPlugin.java
@@ -47,7 +47,7 @@ public class AvroFormatPlugin extends EasyFormatPlugin<AvroFormatConfig> {
   }
 
   public AvroFormatPlugin(String name, DrillbitContext context, Configuration fsConf, StoragePluginConfig config, AvroFormatConfig formatPluginConfig) {
-    super(name, context, fsConf, config, formatPluginConfig, true, false, false, false, Lists.newArrayList("avro"), "avro");
+    super(name, context, fsConf, config, formatPluginConfig, true, false, true, false, Lists.newArrayList("avro"), "avro");
   }
 
   @Override
@@ -57,7 +57,7 @@ public class AvroFormatPlugin extends EasyFormatPlugin<AvroFormatConfig> {
 
   @Override
   public RecordReader getRecordReader(FragmentContext context, DrillFileSystem dfs, FileWork fileWork, List<SchemaPath> columns) throws ExecutionSetupException {
-    return new AvroRecordReader(context, fileWork.getPath(), dfs, columns);
+    return new AvroRecordReader(context, fileWork.getPath(), fileWork.getStart(), fileWork.getLength(), dfs, columns);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/8f4ca6ee/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
index a52fd22..271c8e9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/avro/AvroRecordReader.java
@@ -65,6 +65,8 @@ public class AvroRecordReader extends AbstractRecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AvroRecordReader.class);
 
   private final Path hadoop;
+  private final long start;
+  private final long end;
   private DrillBuf buffer;
   private VectorContainerWriter writer;
 
@@ -77,17 +79,23 @@ public class AvroRecordReader extends AbstractRecordReader {
 
   public AvroRecordReader(final FragmentContext fragmentContext,
                           final String inputPath,
+                          final long start,
+                          final long length,
                           final FileSystem fileSystem,
                           final List<SchemaPath> projectedColumns) {
-    this(fragmentContext, inputPath, fileSystem, projectedColumns, DEFAULT_BATCH_SIZE);
+    this(fragmentContext, inputPath, start, length, fileSystem, projectedColumns, DEFAULT_BATCH_SIZE);
   }
 
   public AvroRecordReader(final FragmentContext fragmentContext,
                           final String inputPath,
+                          final long start,
+                          final long length,
                           final FileSystem fileSystem,
                           List<SchemaPath> projectedColumns, final int defaultBatchSize) {
 
     hadoop = new Path(inputPath);
+    this.start = start;
+    this.end = start + length;
     buffer = fragmentContext.getManagedBuffer();
     this.fs = fileSystem;
 
@@ -101,6 +109,8 @@ public class AvroRecordReader extends AbstractRecordReader {
 
     try {
       reader = new DataFileReader<>(new FsInput(hadoop, fs.getConf()), new GenericDatumReader<GenericContainer>());
+      logger.debug("Processing file : {}, start position : {}, end position : {} ", hadoop, start, end);
+      reader.sync(this.start);
     } catch (IOException e) {
       throw new ExecutionSetupException(e);
     }
@@ -125,7 +135,7 @@ public class AvroRecordReader extends AbstractRecordReader {
 
       // XXX - Implement batch size
 
-      for (GenericContainer container = null; reader.hasNext(); recordCount++) {
+      for (GenericContainer container = null; reader.hasNext() && !reader.pastSync(end); recordCount++) {
         writer.setPosition(recordCount);
         container = reader.next(container);
         processRecord(container, container.getSchema());


[02/15] drill git commit: DRILL-3180: Initial JDBC plugin implementation.

Posted by ja...@apache.org.
DRILL-3180: Initial JDBC plugin implementation.


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

Branch: refs/heads/master
Commit: 8478e9fb1d7e6881d8f092ae9ff3a338f2e023a6
Parents: b525692
Author: MPierre <mp...@mac.com>
Authored: Sat Aug 1 18:07:18 2015 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sun Sep 13 17:02:18 2015 -0700

----------------------------------------------------------------------
 contrib/storage-mpjdbc/pom.xml                  |  79 ++++
 .../exec/store/mpjdbc/MPJdbcBatchCreator.java   |  54 +++
 .../drill/exec/store/mpjdbc/MPJdbcClient.java   | 300 ++++++++++++
 .../exec/store/mpjdbc/MPJdbcClientOptions.java  |  52 ++
 .../exec/store/mpjdbc/MPJdbcCnxnManager.java    |  69 +++
 .../exec/store/mpjdbc/MPJdbcFilterBuilder.java  | 235 +++++++++
 .../exec/store/mpjdbc/MPJdbcFilterRule.java     |  60 +++
 .../exec/store/mpjdbc/MPJdbcFormatConfig.java   | 109 +++++
 .../exec/store/mpjdbc/MPJdbcFormatPlugin.java   | 170 +++++++
 .../exec/store/mpjdbc/MPJdbcGroupScan.java      | 181 +++++++
 .../exec/store/mpjdbc/MPJdbcRecordReader.java   | 471 +++++++++++++++++++
 .../drill/exec/store/mpjdbc/MPJdbcScanSpec.java |  76 +++
 .../exec/store/mpjdbc/MPJdbcSchemaConfig.java   |  80 ++++
 .../exec/store/mpjdbc/MPJdbcSchemaFilter.java   |  23 +
 .../exec/store/mpjdbc/MPJdbcSchemaSubScan.java  |  55 +++
 .../drill/exec/store/mpjdbc/MPJdbcSubScan.java  | 119 +++++
 .../resources/bootstrap-storage-plugins.json    |  12 +
 .../src/main/resources/checkstyle-config.xml    |  41 ++
 .../main/resources/checkstyle-suppressions.xml  |  19 +
 .../src/main/resources/drill-module.conf        |  30 ++
 20 files changed, 2235 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/pom.xml
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/pom.xml b/contrib/storage-mpjdbc/pom.xml
new file mode 100755
index 0000000..5e9afca
--- /dev/null
+++ b/contrib/storage-mpjdbc/pom.xml
@@ -0,0 +1,79 @@
+<?xml version="1.0"?>
+<!--
+ Licensed to the Apache Software Foundation (ASF) under one or more
+ contributor license agreements.  See the NOTICE file distributed with
+ this work for additional information regarding copyright ownership.
+ The ASF licenses this file to You under the Apache License, Version 2.0
+ (the "License"); you may not use this file except in compliance with
+ the License.  You may obtain a copy of the License at
+
+     http://www.apache.org/licenses/LICENSE-2.0
+
+ Unless required by applicable law or agreed to in writing, software
+ distributed under the License is distributed on an "AS IS" BASIS,
+ WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ See the License for the specific language governing permissions and
+ limitations under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd" xmlns="http://maven.apache.org/POM/4.0.0"
+    xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+  <modelVersion>4.0.0</modelVersion>
+  <parent>
+    <artifactId>drill-contrib-parent</artifactId>
+    <groupId>org.apache.drill.contrib</groupId>
+    <version>0.9.0-SNAPSHOT</version>
+  </parent>
+
+  <artifactId>drill-mpjdbc-storage</artifactId>
+
+  <name>contrib/mpjdbc-storage-plugin</name>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <version>${project.version}</version>
+    </dependency>
+    
+
+    <!-- Test dependencie -->
+    <dependency>
+      <groupId>org.apache.drill.exec</groupId>
+      <artifactId>drill-java-exec</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.drill</groupId>
+      <artifactId>drill-common</artifactId>
+      <classifier>tests</classifier>
+      <version>${project.version}</version>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.yammer.metrics</groupId>
+      <artifactId>metrics-core</artifactId>
+      <version>2.1.1</version>
+      <scope>test</scope>
+    </dependency>
+  </dependencies>
+
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+          <systemProperties>
+            <property>
+              <name>logback.log.dir</name>
+              <value>${project.build.directory}/surefire-reports</value>
+            </property>
+          </systemProperties>
+        </configuration>
+      </plugin>
+    </plugins>
+  </build>
+  
+</project>

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
new file mode 100644
index 0000000..af08b2e
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcBatchCreator.java
@@ -0,0 +1,54 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.impl.BatchCreator;
+import org.apache.drill.exec.physical.impl.ScanBatch;
+//import org.apache.drill.exec.record.CloseableRecordBatch;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.mpjdbc.MPJdbcSchemaSubScan;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+
+public class MPJdbcBatchCreator implements BatchCreator<MPJdbcSubScan> {
+  @Override
+  public RecordBatch getBatch(FragmentContext context, MPJdbcSubScan config,
+      List<RecordBatch> children) throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    List<RecordReader> readers = Lists.newArrayList();
+    List<SchemaPath> columns = null;
+    try {
+      if ((columns = config.getColumns()) == null) {
+        columns = GroupScan.ALL_COLUMNS;
+      }
+      readers.add(new MPJdbcRecordReader(context,config));
+    } catch (Exception e1) {
+      throw new ExecutionSetupException(e1);
+    }
+    return new ScanBatch(config, context, readers.iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
new file mode 100644
index 0000000..f3bf81d
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClient.java
@@ -0,0 +1,300 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Schema.TableType;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.linq4j.Extensions;
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.exec.store.AbstractSchema;
+
+import com.google.common.collect.ImmutableList;
+
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DynamicDrillTable;
+
+class MPJdbcClient {
+    static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
+       .getLogger(MPJdbcClient.class);
+
+    private MPJdbcClientOptions clientOptions;
+    private Connection conn;
+    private DatabaseMetaData metadata;
+    private String uri;
+    private OdbcSchema defaultSchema;
+    private MPJdbcFormatPlugin plugin;
+    private String plugName;
+
+    public MPJdbcClient(String uri, MPJdbcClientOptions clientOptions,
+            MPJdbcFormatPlugin plugin) {
+        try {
+            Class.forName(clientOptions.getDriver()).newInstance();
+            this.clientOptions = clientOptions;
+
+            String user = this.clientOptions.getUser();
+            String passwd = this.clientOptions.getPassword();
+            this.plugin = plugin;
+            this.uri = uri;
+
+            if (user == null || user.length() == 0 || passwd.length() == 0) {
+                logger.info("username, password assumed to be in the uri");
+                this.conn = DriverManager.getConnection(uri);
+            } else {
+                this.conn = DriverManager.getConnection(uri, user, passwd);
+            }
+            this.metadata = this.conn.getMetaData();
+            this.plugName = plugin.getName();
+        } catch (InstantiationException e) {
+            // TODO Auto-generated catch block
+           new DrillRuntimeException(e);
+        } catch (IllegalAccessException e) {
+            // TODO Auto-generated catch block
+            new DrillRuntimeException(e);
+        } catch (ClassNotFoundException e) {
+            // TODO Auto-generated catch block
+            new DrillRuntimeException(e);
+        } catch (SQLException e) {
+            new DrillRuntimeException(e);
+        }
+    }
+
+    public Connection getConnection() {
+        return this.conn;
+    }
+
+    public Map<String, Integer> getSchemas() {
+        Map<String, Integer> lst = new HashMap<String, Integer>();
+        try {
+            ResultSet rs = this.metadata.getCatalogs();
+            while (rs.next()) {
+                Integer val = lst.get(rs.getString(1));
+                if (val == null) {
+                    lst.put(rs.getString(1), new Integer(1));
+                }
+            }
+
+        } catch (SQLException e) {
+            new DrillRuntimeException(e);
+        }
+        return lst;
+    }
+
+    public Set<String> getTables(String catalog) {
+        Set<String> lst = new HashSet<String>();
+
+        String[] typeList = { "TABLE", "VIEW" };
+        try {
+            ResultSet rs = this.metadata
+                    .getTables(catalog,null, null, null);
+            while (rs.next()) {
+                if (rs.getString(1) != null) {
+                  //lst.add(rs.getString(1) + "." + rs.getString("TABLE_NAME"));
+                  lst.add(rs.getString("TABLE_NAME"));
+                } else {
+                    lst.add(rs.getString("TABLE_NAME"));
+                }
+            }
+
+        } catch (SQLException e) {
+            throw new DrillRuntimeException(e);
+        }
+        return lst;
+    }
+
+    public List<String> getDatabases() {
+        List<String> lst = new ArrayList<String>();
+        try {
+            ResultSet rs = this.metadata.getCatalogs();
+            while (rs.next()) {
+                lst.add(rs.getString(0));
+            }
+        } catch (SQLException e) {
+            e.printStackTrace();
+        }
+        return lst;
+    }
+
+    public void close() {
+        // TODO Auto-generated method stub
+        try {
+            this.conn.close();
+        } catch (SQLException e) {
+            e.printStackTrace();
+        }
+
+    }
+
+    public OdbcSchema getSchema() {
+        List<String> l = new ArrayList<String>();
+        String currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                plugin).getCurrentSchema();
+        defaultSchema = new OdbcSchema(currentSchema);
+        return defaultSchema;
+    }
+
+    public OdbcSchema getSchema(String name) {
+        List<String> l = new ArrayList<String>();
+        OdbcSchema schema = new OdbcSchema(name);
+        return schema;
+    }
+
+    public class OdbcSchema extends AbstractSchema {
+
+        private Map<String, Integer> sub_schemas;
+        private String currentSchema;
+        private Set<String> tables;
+
+        public OdbcSchema(String name) {
+            super(ImmutableList.<String> of(), name);
+            /*currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                    plugin).getCurrentSchema();
+            if (currentSchema == null) {
+                currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                        plugin).getCurrentSchema();
+            }
+            */
+            if(name.equals("")) {
+              sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
+                  .getSchemas();
+            }
+            tables = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
+                    .getTables(name);
+        }
+
+        public OdbcSchema(List<String> parentSchemaPath, String name) {
+            super(parentSchemaPath, name);
+            currentSchema = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                    plugin).getCurrentSchema();
+            if (currentSchema == null) {
+                currentSchema = "ROOT";
+            }
+            sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
+                    .getSchemas();
+            // TODO Auto-generated constructor stub
+        }
+
+        @Override
+        public String getTypeName() {
+            // TODO Auto-generated method stub
+            return "odbc";
+        }
+
+        @Override
+        public AbstractSchema getSubSchema(String name) {
+            if (sub_schemas == null) {
+                sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                        plugin).getSchemas();
+            }
+            Integer a = sub_schemas.get(name);
+            if (a == 1) {
+                return new OdbcSchema(name);
+            }
+            return null;
+        }
+
+        @Override
+        public Table getTable(String name) {
+            // TODO Auto-generated method stub
+          String tableName = null;
+          if(name.contains(".")) {
+            String[] val = name.split("\\.");
+            OdbcSchema sub = (OdbcSchema) this.getSubSchema(val[0]);
+            return sub.getTable(val[1]);
+          }
+          Iterator<String> iter = tables.iterator();
+          while(iter.hasNext()) {
+            tableName = iter.next();
+            if(tableName.equalsIgnoreCase(name)) {
+              break;
+            }
+            else {
+              tableName = null;
+            }
+          }
+          if(tableName == null) {
+            return null;
+          }
+          MPJdbcScanSpec spec = new MPJdbcScanSpec(this.name, tableName,"");
+          return new DynamicDrillTable(plugin, "odbc", spec);
+        }
+
+        @Override
+        public Set<String> getTableNames() {
+            // TODO Auto-generated method stub
+            Set<String> Tables = MPJdbcCnxnManager.getClient(uri, clientOptions,
+                    plugin).getTables(name);
+            return Tables;
+        }
+
+        @Override
+        public Set<String> getSubSchemaNames() {
+            // TODO Auto-generated method stub
+            sub_schemas = MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
+                    .getSchemas();
+            return sub_schemas.keySet();
+        }
+
+        @Override
+        public Collection<Function> getFunctions(String name) {
+            // TODO Auto-generated method stub
+            return super.getFunctions(name);
+        }
+
+        @Override
+        public AbstractSchema getDefaultSchema() {
+            return MPJdbcCnxnManager.getClient(uri, clientOptions, plugin)
+                    .getDefaultSchema();
+        }
+
+    }
+
+    public String getCurrentSchema() {
+        // TODO Auto-generated method stub
+        try {
+            return this.conn.getCatalog();
+        } catch (SQLException e) {
+            // TODO Auto-generated catch block
+            e.printStackTrace();
+        }
+        return null;
+    }
+
+    public AbstractSchema getDefaultSchema() {
+        // TODO Auto-generated method stub
+        return defaultSchema;
+    }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
new file mode 100644
index 0000000..84b6348
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcClientOptions.java
@@ -0,0 +1,52 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+public class MPJdbcClientOptions {
+  private String driver;
+  private String user;
+  private String passwd;
+
+  public MPJdbcClientOptions(String driver, String user, String passwd) {
+    this.driver = driver;
+    this.user = user;
+    this.passwd = passwd;
+  }
+
+  public MPJdbcClientOptions(MPJdbcFormatConfig storageConfig) {
+    this.driver = storageConfig.getDriver();
+    this.user = storageConfig.getUser();
+    this.passwd = storageConfig.getPasswd();
+  }
+
+  public String getDriver() {
+    // TODO Auto-generated method stub
+    return this.driver;
+  }
+
+  public String getUser() {
+    // TODO Auto-generated method stub
+    return this.user;
+  }
+
+  public String getPassword() {
+    // TODO Auto-generated method stub
+    return this.passwd;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
new file mode 100644
index 0000000..7e43c32
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcCnxnManager.java
@@ -0,0 +1,69 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.concurrent.TimeUnit;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+
+import org.apache.drill.exec.store.mpjdbc.MPJdbcClientOptions;
+
+public class MPJdbcCnxnManager {
+
+  private static final Logger logger = LoggerFactory
+      .getLogger(MPJdbcCnxnManager.class);
+  private static Cache<String, MPJdbcClient> uriClientMap;
+
+  static {
+    uriClientMap = CacheBuilder.newBuilder().maximumSize(5)
+        .expireAfterAccess(10, TimeUnit.MINUTES)
+        .removalListener(new uriCloser()).build();
+  }
+
+  private static class uriCloser implements RemovalListener<String, MPJdbcClient> {
+
+    @Override
+    public synchronized void onRemoval(
+        RemovalNotification<String, MPJdbcClient> removal) {
+      removal.getValue().close();
+      logger.debug("Closed connection to {}.", removal.getKey().toString());
+    }
+
+  }
+
+  public synchronized static MPJdbcClient getClient(String uri,
+      MPJdbcClientOptions clientOptions, MPJdbcFormatPlugin plugin) {
+    MPJdbcClient client = uriClientMap.getIfPresent(uri);
+    if (client == null) {
+      client = new MPJdbcClient(uri, clientOptions,plugin);
+      if (client.getConnection() != null) {
+        uriClientMap.put(uri, client);
+      } else {
+        return null;
+      }
+    }
+
+    return client;
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.java
new file mode 100644
index 0000000..488a55d
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterBuilder.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.drill.exec.store.mpjdbc;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.BooleanOperator;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
+import org.apache.drill.exec.store.mpjdbc.MPJdbcScanSpec;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.ImmutableList;
+
+public class MPJdbcFilterBuilder extends
+    AbstractExprVisitor<MPJdbcScanSpec, Void, RuntimeException> {
+  static final Logger logger = LoggerFactory
+      .getLogger(MPJdbcFilterBuilder.class);
+  final MPJdbcGroupScan groupScan;
+  final LogicalExpression le;
+  private boolean allExpressionsConverted = true;
+
+  public MPJdbcFilterBuilder(MPJdbcGroupScan groupScan,
+      LogicalExpression conditionExp) {
+    this.groupScan = groupScan;
+    this.le = conditionExp;
+  }
+
+  public MPJdbcScanSpec parseTree() {
+    MPJdbcScanSpec parsedSpec = le.accept(this, null);
+    if (parsedSpec != null) {
+      parsedSpec = mergeScanSpecs("booleanAnd", this.groupScan.getScanSpec(),
+          parsedSpec);
+    }
+    return parsedSpec;
+  }
+
+  private MPJdbcScanSpec mergeScanSpecs(String functionName,
+     MPJdbcScanSpec leftScanSpec, MPJdbcScanSpec rightScanSpec) {
+    List<String> newFilter;
+    switch (functionName) {
+    case "booleanAnd":
+      if (leftScanSpec.getFilters() != null
+          && rightScanSpec.getFilters() != null) {
+        /* newFilter = MongoUtils.andFilterAtIndex(leftScanSpec.getFilters(),
+            rightScanSpec.getFilters()); */
+      } else if (leftScanSpec.getFilters() != null) {
+        newFilter = leftScanSpec.getFilters();
+      } else {
+        newFilter = rightScanSpec.getFilters();
+      }
+      break;
+    case "booleanOr":
+     /* newFilter = OdbcUtils.orFilterAtIndex(leftScanSpec.getFilters(),
+          rightScanSpec.getFilters()); */
+    }
+    MPJdbcScanSpec mp =  new MPJdbcScanSpec(groupScan.getScanSpec().getDatabase(), groupScan
+        .getScanSpec().getTable(), groupScan.getScanSpec().getColumns());
+    return mp;
+  }
+
+  public boolean isAllExpressionsConverted() {
+    return allExpressionsConverted;
+  }
+
+  @Override
+  public MPJdbcScanSpec visitUnknown(LogicalExpression e, Void value)
+      throws RuntimeException {
+    allExpressionsConverted = false;
+    return null;
+  }
+
+  @Override
+  public MPJdbcScanSpec visitBooleanOperator(BooleanOperator op, Void value) {
+    List<LogicalExpression> args = op.args;
+    MPJdbcScanSpec nodeScanSpec = null;
+    String functionName = op.getName();
+    for (int i = 0; i < args.size(); ++i) {
+      switch (functionName) {
+      case "booleanAnd":
+      case "booleanOr":
+        if (nodeScanSpec == null) {
+          nodeScanSpec = args.get(i).accept(this, null);
+        } else {
+          MPJdbcScanSpec scanSpec = args.get(i).accept(this, null);
+          if (scanSpec != null) {
+            nodeScanSpec = mergeScanSpecs(functionName, nodeScanSpec, scanSpec);
+          } else {
+            allExpressionsConverted = false;
+          }
+        }
+        break;
+      }
+    }
+    return nodeScanSpec;
+  }
+
+  @Override
+  public MPJdbcScanSpec visitFunctionCall(FunctionCall call, Void value)
+      throws RuntimeException {
+    MPJdbcScanSpec nodeScanSpec = null;
+    String functionName = call.getName();
+    ImmutableList<LogicalExpression> args = call.args;
+    LogicalExpression nameVal = call.args.get(0);
+    LogicalExpression valueVal = null;
+    StringBuilder strBuilder = new StringBuilder();
+    if(call.args.size() >= 2) {
+      valueVal = call.args.get(1);
+    }
+    logger.info("Name Val:" + nameVal.toString());
+    logger.info("Value Val:" + valueVal.toString());
+
+    switch(functionName) {
+    case "equal":
+      break;
+     default:
+       break;
+    }
+    /*
+    if (OdbcCompareFunctionProcessor.isCompareFunction(functionName)) {
+      OdbcCompareFunctionProcessor processor = OdbcCompareFunctionProcessor
+          .process(call);
+      if (processor.isSuccess()) {
+        try {
+          nodeScanSpec = createOdbcScanSpec(processor.getFunctionName(),
+              processor.getPath(), processor.getValue());
+        } catch (Exception e) {
+          logger.error(" Failed to creare Filter ", e);
+          // throw new RuntimeException(e.getMessage(), e);
+        }
+      }
+    } else {
+      switch (functionName) {
+      case "booleanAnd":
+      case "booleanOr":
+        MPJdbcScanSpec leftScanSpec = args.get(0).accept(this, null);
+        MPJdbcScanSpec rightScanSpec = args.get(1).accept(this, null);
+        if (leftScanSpec != null && rightScanSpec != null) {
+          nodeScanSpec = mergeScanSpecs(functionName, leftScanSpec,
+              rightScanSpec);
+        } else {
+          allExpressionsConverted = false;
+          if ("booleanAnd".equals(functionName)) {
+            nodeScanSpec = leftScanSpec == null ? rightScanSpec : leftScanSpec;
+          }
+        }
+        break;
+      }
+    }
+ */
+    if (nodeScanSpec == null) {
+      allExpressionsConverted = false;
+    }
+
+    return nodeScanSpec;
+  }
+
+  private MPJdbcScanSpec createOdbcScanSpec(String functionName,
+      SchemaPath field, Object fieldValue) throws ClassNotFoundException,
+      Exception {
+    // extract the field name
+    String fieldName = field.getAsUnescapedPath();
+    /*
+    OdbcCompareOp compareOp = null;
+    switch (functionName) {
+    case "equal":
+      compareOp = OdbcCompareOp.EQUAL;
+      break;
+    case "not_equal":
+      compareOp = OdbcCompareOp.NOT_EQUAL;
+      break;
+    case "greater_than_or_equal_to":
+      compareOp = OdbcCompareOp.GREATER_OR_EQUAL;
+      break;
+    case "greater_than":
+      compareOp = OdbcCompareOp.GREATER;
+      break;
+    case "less_than_or_equal_to":
+      compareOp = OdbcCompareOp.LESS_OR_EQUAL;
+      break;
+    case "less_than":
+      compareOp = OdbcCompareOp.LESS;
+      break;
+    case "isnull":
+    case "isNull":
+    case "is null":
+      compareOp = OdbcCompareOp.IFNULL;
+      break;
+    case "isnotnull":
+    case "isNotNull":
+    case "is not null":
+      compareOp = OdbcCompareOp.IFNOTNULL;
+      break;
+    }
+
+    if (compareOp != null) {
+      BasicDBObject queryFilter = new BasicDBObject();
+      if (compareOp == OdbcCompareOp.IFNULL) {
+        queryFilter.put(fieldName,
+            new BasicDBObject(OdbcCompareOp.EQUAL.getCompareOp(), null));
+      } else if (compareOp == OdbcCompareOp.IFNOTNULL) {
+        queryFilter.put(fieldName,
+            new BasicDBObject(OdbcCompareOp.NOT_EQUAL.getCompareOp(), null));
+      } else {
+        queryFilter.put(fieldName, new BasicDBObject(compareOp.getCompareOp(),
+            fieldValue));
+      }
+      return new MPJdbcScanSpec(groupScan.getScanSpec().getDbName(), groupScan
+          .getScanSpec().getCollectionName(), queryFilter);
+    }
+    */
+    return null;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
new file mode 100644
index 0000000..c0564d0
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFilterRule.java
@@ -0,0 +1,60 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.Iterator;
+import java.util.List;
+
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.plan.RelOptRuleOperand;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.rex.RexNode;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.FilterPrel;
+import org.apache.drill.exec.planner.physical.ScanPrel;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+
+public class MPJdbcFilterRule extends StoragePluginOptimizerRule {
+  public static final StoragePluginOptimizerRule INSTANCE = new MPJdbcFilterRule();
+  public MPJdbcFilterRule(RelOptRuleOperand operand, String description) {
+    super(operand, description);
+    // TODO Auto-generated constructor stub
+  }
+
+  public MPJdbcFilterRule() {
+    // TODO Auto-generated constructor stub
+    super(RelOptHelper.some(FilterPrel.class, RelOptHelper.any(ScanPrel.class)), "MPJdbcFilterRule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    // TODO Auto-generated method stub
+    final ScanPrel scan = (ScanPrel) call.rel(1);
+    final FilterPrel filter = (FilterPrel) call.rel(0);
+    final RexNode condition = filter.getCondition();
+    final LogicalExpression conditionExp = DrillOptiq.toDrill(new DrillParseContext(), scan, condition);
+    MPJdbcGroupScan grpScan= (MPJdbcGroupScan) scan.getGroupScan();
+    MPJdbcFilterBuilder builder = new MPJdbcFilterBuilder(grpScan,conditionExp);
+    MPJdbcScanSpec result = builder.parseTree();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
new file mode 100644
index 0000000..8edce3e
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatConfig.java
@@ -0,0 +1,109 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import org.apache.drill.common.logical.StoragePluginConfig;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.ObjectMapper;
+
+@JsonTypeName(MPJdbcFormatConfig.NAME)
+public class MPJdbcFormatConfig extends StoragePluginConfig {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
+      .getLogger(MPJdbcFormatConfig.class);
+  public static final String NAME = "jdbc";
+
+  @JsonIgnore
+  private String driver;
+  @JsonIgnore
+  private String uri;
+  @JsonIgnore
+  private String username;
+  @JsonIgnore
+  private String password;
+
+  @JsonCreator
+  public MPJdbcFormatConfig(@JsonProperty("driver") String driver,
+      @JsonProperty("uri") String uri,
+      @JsonProperty("username") String username,
+      @JsonProperty("password") String password) {
+    this.driver = driver == null ? "" : driver;
+    this.uri = uri == null ? "jdbc://" : uri;
+    this.username = username == null ? "" : username;
+    this.password = password == null ? "" : password;
+
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    MPJdbcFormatConfig that = (MPJdbcFormatConfig) o;
+
+    if (uri != null ? !uri.equals(that.uri) : that.uri != null) {
+      return false;
+    }
+
+    return true;
+  }
+
+  @JsonProperty("driver")
+  public String getDriver() {
+    return this.driver;
+  }
+
+  @JsonProperty("uri")
+  public String getUri() {
+    return this.uri;
+  }
+
+  @JsonProperty("username")
+  public String getUser() {
+    return this.username;
+  }
+
+  @JsonProperty("password")
+  public String getPasswd() {
+    return this.password;
+  }
+
+  @Override
+  public int hashCode() {
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      String outval = mapper.writeValueAsString(this);
+      logger.info("FormatConfigHashCode:" + outval);
+
+      return outval.hashCode();
+    } catch (JsonProcessingException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+      return uri.hashCode();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
new file mode 100644
index 0000000..5a0dd4b
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcFormatPlugin.java
@@ -0,0 +1,170 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.io.IOException;
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+
+import org.apache.calcite.schema.Table;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.schema.Schema;
+import org.apache.calcite.schema.SchemaPlus;
+import org.apache.calcite.schema.Statistic;
+import org.apache.calcite.schema.Schema.TableType;
+import org.apache.calcite.schema.Function;
+import org.apache.calcite.linq4j.Extensions;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.logical.FormatPluginConfig;
+import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.AbstractWriter;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.AbstractStoragePlugin;
+import org.apache.drill.exec.store.SchemaConfig;
+import org.apache.drill.exec.store.StoragePluginOptimizerRule;
+import org.apache.drill.exec.store.dfs.DrillFileSystem;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP;
+import org.apache.drill.exec.store.mock.MockStorageEngine;
+import org.apache.drill.exec.store.mock.MockStorageEngineConfig;
+import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
+import org.apache.drill.exec.store.mpjdbc.MPJdbcClient.OdbcSchema;
+import org.apache.calcite.plan.RelOptRuleCall;
+import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.rel.type.RelDataTypeFactory;
+
+import com.fasterxml.jackson.annotation.JsonTypeName;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.ImmutableSet;
+
+public class MPJdbcFormatPlugin extends AbstractStoragePlugin {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
+      .getLogger(MPJdbcFormatPlugin.class);
+
+  private final MPJdbcFormatConfig storageConfig;
+  protected String name = "odbc";
+  private final DrillbitContext context;
+
+  public MPJdbcFormatPlugin(MPJdbcFormatConfig storageConfig,
+      DrillbitContext context, String name) {
+    this.context = context;
+    this.storageConfig = storageConfig;
+    ObjectMapper mapper = new ObjectMapper();
+    try {
+      String result = mapper.writeValueAsString(storageConfig);
+      logger.info(result);
+    } catch (JsonProcessingException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+    this.name = name;
+  }
+
+  @Override
+  public void registerSchemas(SchemaConfig config, SchemaPlus parent) {
+    if(storageConfig == null) {
+       logger.info("StorageConfig is null");
+    }
+    MPJdbcClientOptions options = new MPJdbcClientOptions(storageConfig);
+    MPJdbcClient client = MPJdbcCnxnManager.getClient(storageConfig.getUri(),
+        options,this);
+    Connection conn = (client == null) ? null : client.getConnection();
+    Map<String, Integer> schemas;
+    if(client == null) {
+      logger.info("Could not create client...");
+    }
+    OdbcSchema o = client.getSchema();
+    SchemaPlus tl = parent.add(this.name, o);
+    try {
+      schemas = client.getSchemas();
+      Set<Entry<String, Integer>> a = schemas.entrySet();
+      Iterator<Entry<String, Integer>> aiter = a.iterator();
+      while (aiter.hasNext()) {
+        Entry<String, Integer> val = aiter.next();
+        String catalog = val.getKey();
+        OdbcSchema sc = client.getSchema(catalog);
+        tl.add(catalog, sc);
+      }
+    } catch (Exception e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public MPJdbcFormatConfig getConfig() {
+    logger.info("MPJdbcFormatPlugin:getConfig called");
+    logger.info(storageConfig.toString());
+    return storageConfig;
+  }
+
+  public DrillbitContext getContext() {
+    return this.context;
+  }
+
+  public String getName() {
+      return this.name;
+  }
+
+  @Override
+  public boolean supportsRead() {
+    return true;
+  }
+/*
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName,JSONOptions selection)
+      throws IOException {
+    MPJdbcScanSpec odbcScanSpec = selection.getListWith(new ObjectMapper(),
+        new TypeReference<MPJdbcScanSpec>() {
+        });
+    return new MPJdbcGroupScan(userName,this, odbcScanSpec, null);
+  }
+  */
+  @Override
+  public AbstractGroupScan getPhysicalScan(String userName,JSONOptions selection,List<SchemaPath> columns)
+      throws IOException {
+    MPJdbcScanSpec mPJdbcScanSpec = selection.getListWith(new ObjectMapper(),
+        new TypeReference<MPJdbcScanSpec>() {
+        });
+    return new MPJdbcGroupScan(userName,this, mPJdbcScanSpec, columns);
+  }
+
+  @Override
+  public Set<StoragePluginOptimizerRule> getOptimizerRules() {
+    // TODO Auto-generated method stub
+    return ImmutableSet.of(MPJdbcFilterRule.INSTANCE);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
new file mode 100644
index 0000000..a16f8c8
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcGroupScan.java
@@ -0,0 +1,181 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Set;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.ScanStats;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.schedule.CompleteFileWork;
+import org.apache.drill.common.expression.SchemaPath;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.ListMultimap;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+
+public class MPJdbcGroupScan extends AbstractGroupScan {
+
+  private MPJdbcFormatPlugin plugin;
+  private MPJdbcFormatConfig pluginConfig;
+  private MPJdbcScanSpec mPJdbcScanSpec;
+  private List<SchemaPath> columns;
+  private String userName;
+  private Map<Integer, List<MPJdbcScanSpec>> endpointFragmentMapping;
+
+  public MPJdbcGroupScan(String userName,MPJdbcFormatPlugin storagePlugin, MPJdbcScanSpec scanSpec,
+      List<SchemaPath> columns) {
+    super(userName);
+    this.plugin = storagePlugin;
+    this.pluginConfig = storagePlugin.getConfig();
+    this.mPJdbcScanSpec = scanSpec;
+    this.userName = userName;
+    this.columns = columns == null || columns.size() == 0 ? ALL_COLUMNS
+        : columns;
+  }
+
+  public MPJdbcGroupScan(MPJdbcGroupScan that) {
+    super(that);
+    this.columns = that.columns;
+    this.plugin = that.plugin;
+    this.endpointFragmentMapping = that.endpointFragmentMapping;
+    this.pluginConfig = that.pluginConfig;
+    this.mPJdbcScanSpec = that.mPJdbcScanSpec;
+  }
+
+  @Override
+  public SubScan getSpecificScan(int minorFragmentId)
+      throws ExecutionSetupException {
+    // TODO Auto-generated method stub
+    return new MPJdbcSubScan(plugin,userName, pluginConfig,
+        endpointFragmentMapping.get(minorFragmentId), columns);
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    // TODO Auto-generated method stub
+    return -1;
+  }
+
+  @Override
+  public String getDigest() {
+    // TODO Auto-generated method stub
+    return toString();
+  }
+
+  @Override
+  public GroupScan clone(List<SchemaPath> columns) {
+    MPJdbcGroupScan newScan = new MPJdbcGroupScan(userName,plugin, mPJdbcScanSpec, columns);
+    return newScan;
+
+  }
+
+  @Override
+  public ScanStats getScanStats() {
+    // TODO Auto-generated method stub
+    return ScanStats.TRIVIAL_TABLE;
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children)
+      throws ExecutionSetupException {
+    Preconditions.checkArgument(children.isEmpty());
+    return new MPJdbcGroupScan(this);
+    // TODO Auto-generated method stub
+  }
+  @Override
+  @JsonIgnore
+  public boolean canPushdownProjects(List<SchemaPath> columns) {
+    this.columns = columns;
+    return true;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    Map<String, DrillbitEndpoint> endpointMap = new HashMap<String, DrillbitEndpoint>();
+    for (DrillbitEndpoint ep : plugin.getContext().getBits()) {
+      endpointMap.put(ep.getAddress(), ep);
+    }
+
+    Map<DrillbitEndpoint, EndpointAffinity> affinityMap = new HashMap<DrillbitEndpoint, EndpointAffinity>();
+    DrillbitEndpoint ep = endpointMap.get(plugin.getConfig().getUri());
+    if (ep != null) {
+      EndpointAffinity affinity = affinityMap.get(ep);
+      if (affinity == null) {
+        affinityMap.put(ep, new EndpointAffinity(ep, 1));
+      } else {
+        affinity.addAffinity(1);
+      }
+    }
+    return Lists.newArrayList(affinityMap.values());
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> incomingEndpoints) {
+    final int numSlots = incomingEndpoints.size();
+    int totalAssignmentsTobeDone = 1;
+    Preconditions.checkArgument(numSlots <= totalAssignmentsTobeDone, String
+        .format("Incoming endpoints %d is greater than number of chunks %d",
+            numSlots, totalAssignmentsTobeDone));
+    final int minPerEndpointSlot = (int) Math
+        .floor((double) totalAssignmentsTobeDone / numSlots);
+    final int maxPerEndpointSlot = (int) Math
+        .ceil((double) totalAssignmentsTobeDone / numSlots);
+    /* Map for (index,endpoint)'s */
+    endpointFragmentMapping = Maps.newHashMapWithExpectedSize(numSlots);
+    /* Reverse mapping for above indexes */
+    Map<String, Queue<Integer>> endpointHostIndexListMap = Maps.newHashMap();
+    /*
+     * Initialize these two maps
+     */
+    for (int i = 0; i < numSlots; ++i) {
+      List<MPJdbcScanSpec> val = new ArrayList<MPJdbcScanSpec>(maxPerEndpointSlot);
+      val.add(this.mPJdbcScanSpec);
+      endpointFragmentMapping.put(i, val);
+      String hostname = incomingEndpoints.get(i).getAddress();
+      Queue<Integer> hostIndexQueue = endpointHostIndexListMap.get(hostname);
+      if (hostIndexQueue == null) {
+        hostIndexQueue = Lists.newLinkedList();
+        endpointHostIndexListMap.put(hostname, hostIndexQueue);
+      }
+      hostIndexQueue.add(i);
+    }
+  }
+
+  public MPJdbcScanSpec getScanSpec() {
+    return this.mPJdbcScanSpec;
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
new file mode 100644
index 0000000..498b7fd
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcRecordReader.java
@@ -0,0 +1,471 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import org.apache.drill.common.exceptions.DrillRuntimeException;
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.expression.SchemaPath;
+import org.apache.drill.common.types.TypeProtos;
+import org.apache.drill.common.types.TypeProtos.DataMode;
+import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.expr.holders.DateHolder;
+import org.apache.drill.exec.expr.holders.Decimal38DenseHolder;
+import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.memory.OutOfMemoryException;
+import org.apache.drill.exec.ops.FragmentContext;
+import org.apache.drill.exec.ops.OperatorContext;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.record.MaterializedField.Key;
+import org.apache.drill.exec.store.AbstractRecordReader;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.DateVector;
+import org.apache.drill.exec.vector.Decimal38DenseVector;
+import org.apache.drill.exec.vector.FixedWidthVector;
+import org.apache.drill.exec.vector.Float8Vector;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+import org.apache.drill.exec.vector.NullableDateVector;
+import org.apache.drill.exec.vector.NullableDecimal38DenseVector;
+import org.apache.drill.exec.vector.NullableFloat8Vector;
+import org.apache.drill.exec.vector.NullableIntVector;
+import org.apache.drill.exec.vector.NullableTimeStampVector;
+import org.apache.drill.exec.vector.NullableTimeVector;
+import org.apache.drill.exec.vector.NullableVar16CharVector.Mutator;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+import org.apache.drill.exec.vector.TimeStampVector;
+import org.apache.drill.exec.vector.TimeVector;
+import org.apache.drill.exec.vector.ValueHolderHelper;
+import org.apache.drill.exec.vector.ValueVector;
+import org.apache.drill.exec.vector.VarBinaryVector;
+import org.apache.drill.exec.vector.VarCharVector;
+import org.apache.drill.exec.vector.complex.impl.VectorContainerWriter;
+import org.joda.time.DateTime;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Charsets;
+import com.google.common.collect.Lists;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.DrillBuf;
+
+import java.sql.Connection;
+import java.sql.DatabaseMetaData;
+import java.sql.DriverManager;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Types;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+public class MPJdbcRecordReader extends AbstractRecordReader {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
+      .getLogger(MPJdbcRecordReader.class);
+
+  private ResultSet rec;
+  private VectorContainerWriter writer;
+  private FragmentContext fc;
+  private MPJdbcSubScan scanSpec;
+  private MPJdbcFormatPlugin plugin;
+  private List<MPJdbcScanSpec> scanList;
+  private MPJdbcFormatConfig config;
+  private Connection conn;
+  private Statement statement;
+  private String table;
+  private String database;
+  protected List<ValueVector> vectors = Lists.newArrayList();
+  private int col_cnt = 0;
+  private MajorType.Builder t;
+  private OutputMutator outputMutator;
+  private ResultSetMetaData meta;
+  private OperatorContext operatorContext;
+  private String columns;
+  private List<String> filters;
+
+  public MPJdbcRecordReader(FragmentContext fragmentContext, MPJdbcSubScan scan) {
+    fc = fragmentContext;
+    scanSpec = scan;
+    // TODO Auto-generated constructor stub
+    this.plugin = scanSpec.getPlugin();
+    this.scanList = scanSpec.getScanList();
+    this.config = scanSpec.getConfig();
+    MPJdbcClientOptions options = new MPJdbcClientOptions(config);
+    MPJdbcClient client = MPJdbcCnxnManager.getClient(config.getUri(), options,
+        this.plugin);
+    conn = client.getConnection();
+    Iterator<MPJdbcScanSpec> iter = scanList.iterator();
+    while (iter.hasNext()) {
+      MPJdbcScanSpec o = iter.next();
+      table = o.getTable();
+      database = o.getDatabase();
+      List<SchemaPath> ColList = scan.getColumns();
+      Iterator<SchemaPath> collist_iter = ColList.iterator();
+      StringBuilder b = new StringBuilder();
+      while(collist_iter.hasNext()) {
+          SchemaPath val = collist_iter.next();
+          b.append(val.getAsUnescapedPath().trim());
+          if(collist_iter.hasNext()) {
+              b.append(",");
+          }
+      }
+      columns = b.toString();
+      filters = o.getFilters();
+    }
+    try {
+      statement = conn.createStatement();
+      rec = statement.executeQuery("SELECT " + this.columns + " FROM " + database.trim() + "." + table.trim());
+    } catch (SQLException e) {
+      // TODO Auto-generated catch block
+      e.printStackTrace();
+    }
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    try {
+      meta = rec.getMetaData();
+      col_cnt = meta.getColumnCount();
+      Class cls = null;
+      for (int i = 1; i <= col_cnt; i++) {
+        String column_label = meta.getColumnLabel(i);
+        int types = meta.getColumnType(i);
+        int isnullable = meta.isNullable(i);
+        int width = meta.getPrecision(i);
+        int scale = meta.getScale(i);
+        MaterializedField field = null;
+        switch (types) {
+        case java.sql.Types.BIGINT:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.BIGINT);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableBigIntVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, BigIntVector.class));
+          }
+          break;
+        case Types.DATE:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.DATE);
+          cls = org.apache.drill.exec.vector.DateVector.class;
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableDateVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, DateVector.class));
+          }
+          break;
+        case Types.DECIMAL:
+          t = MajorType.newBuilder().setMinorType(
+              TypeProtos.MinorType.DECIMAL38DENSE);
+          t.setMode(DataMode.OPTIONAL);
+          field = MaterializedField.create(column_label, t.build());
+          vectors.add(output.addField(field, Decimal38DenseVector.class));
+          break;
+        case Types.DOUBLE:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.BIGINT);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableBigIntVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, BigIntVector.class));
+          }
+          break;
+        case Types.FLOAT:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.FLOAT8);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableFloat8Vector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, Float8Vector.class));
+          }
+          break;
+        case Types.INTEGER:
+        case Types.SMALLINT:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.INT);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableIntVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, IntVector.class));
+          }
+          break;
+        case Types.LONGNVARCHAR:
+        case Types.LONGVARCHAR:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
+          t.setWidth(width);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableVarCharVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, VarCharVector.class));
+          }
+          break;
+        case Types.LONGVARBINARY:
+          break;
+        case Types.CHAR:
+        case Types.NCHAR:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
+          t.setWidth(width);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableVarCharVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, VarCharVector.class));
+          }
+          break;
+        case Types.NUMERIC:
+          t = MajorType.newBuilder().setMinorType(
+              TypeProtos.MinorType.DECIMAL38DENSE);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field,
+                NullableDecimal38DenseVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, Decimal38DenseVector.class));
+          }
+          break;
+        case Types.NVARCHAR:
+        case Types.VARCHAR:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
+          t.setWidth(width);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableVarCharVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, VarCharVector.class));
+          }
+          break;
+        case Types.TIME:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.TIME);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableTimeVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, TimeVector.class));
+          }
+          break;
+        case Types.TIMESTAMP:
+          t = MajorType.newBuilder().setMinorType(
+              TypeProtos.MinorType.TIMESTAMP);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableTimeStampVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, TimeStampVector.class));
+          }
+          break;
+        default:
+          t = MajorType.newBuilder().setMinorType(TypeProtos.MinorType.VARCHAR);
+          t.setWidth(width);
+          if (isnullable == 1) {
+            t.setMode(DataMode.OPTIONAL);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, NullableVarCharVector.class));
+          } else {
+            t.setMode(DataMode.REQUIRED);
+            field = MaterializedField.create(column_label, t.build());
+            vectors.add(output.addField(field, VarCharVector.class));
+          }
+          break;
+        }
+      }
+      this.outputMutator = output;
+
+    } catch (SQLException | SchemaChangeException e) {
+      // TODO Auto-generated catch block
+      throw new ExecutionSetupException(e);
+    }
+  }
+
+  @Override
+  public void setOperatorContext(OperatorContext operatorContext) {
+    this.operatorContext = operatorContext;
+  }
+
+  @Override
+  public int next() {
+    // TODO Auto-generated method stub
+    int counter = 0;
+    int pos = 1;
+    int prec = 0;
+    Boolean b = true;
+    try {
+      while (counter < 65536 && b == true) {
+        b = rec.next();
+        if(b == false) {
+            break;
+        }
+        for (ValueVector vv : vectors) {
+          String val = rec.getString(pos);
+          byte[] record = val.getBytes(Charsets.UTF_8);
+          if (vv.getClass().equals(NullableVarCharVector.class)) {
+            NullableVarCharVector v = (NullableVarCharVector) vv;
+            v.getMutator().setSafe(counter, record, 0, record.length);
+            v.getMutator().setValueLengthSafe(counter, record.length);
+          } else if (vv.getClass().equals(VarCharVector.class)) {
+            VarCharVector v = (VarCharVector) vv;
+            v.getMutator().setSafe(counter, record, 0, record.length);
+            v.getMutator().setValueLengthSafe(counter, record.length);
+          } else if (vv.getClass().equals(BigIntVector.class)) {
+            BigIntVector v = (BigIntVector) vv;
+            v.getMutator().setSafe(counter, rec.getLong(pos));
+          } else if (vv.getClass().equals(NullableBigIntVector.class)) {
+            NullableBigIntVector v = (NullableBigIntVector) vv;
+            v.getMutator().setSafe(counter, rec.getLong(pos));
+          } else if (vv.getClass().equals(IntVector.class)) {
+            IntVector v = (IntVector) vv;
+            v.getMutator().setSafe(counter, rec.getInt(pos));
+          } else if (vv.getClass().equals(NullableIntVector.class)) {
+            NullableIntVector v = (NullableIntVector) vv;
+            v.getMutator().setSafe(counter, rec.getInt(pos));
+          } else if (vv.getClass().equals(DateVector.class)) {
+            DateVector v = (DateVector) vv;
+            long dtime = DateTime.parse(val).toDate().getTime(); // DateTime.parse(val).toDateTime().getMillis();
+            v.getMutator().setSafe(counter, dtime);
+          } else if (vv.getClass().equals(NullableDateVector.class)) {
+            NullableDateVector v = (NullableDateVector) vv;
+            if (rec.wasNull()) {
+              v.getMutator().setNull(counter);
+            } else {
+              long dtime = DateTime.parse(val).toDate().getTime();
+              v.getMutator().setSafe(counter, dtime);
+            }
+          } else if (vv.getClass().equals(Decimal38DenseVector.class)) {
+            Decimal38DenseVector v = (Decimal38DenseVector) vv;
+            java.math.BigDecimal d = rec.getBigDecimal(pos);
+          } else if (vv.getClass().equals(NullableDecimal38DenseVector.class)) {
+            NullableDecimal38DenseVector v = (NullableDecimal38DenseVector) vv;
+            java.math.BigDecimal d = rec.getBigDecimal(pos);
+          } else {
+            NullableVarCharVector v = (NullableVarCharVector) vv;
+            v.getMutator().setSafe(counter, record, 0, record.length);
+            v.getMutator().setValueLengthSafe(counter, record.length);
+          }
+          pos++;
+        }
+        pos = 1;
+        counter++;
+      }
+    } catch (SQLException e) {
+      // TODO Auto-generated catch block
+      throw new DrillRuntimeException(e);
+    }
+    //logger.info("Number of rows returned from JDBC: " + counter);
+    for (ValueVector vv : vectors) {
+      vv.getMutator().setValueCount(counter > 0 ? counter : 0);
+    }
+    return counter>0 ? counter : 0;
+  }
+
+  @Override
+  public void allocate(Map<Key, ValueVector> vectorMap)
+      throws OutOfMemoryException {
+    int prec = 0;
+    try {
+      for (ValueVector vv : vectorMap.values()) {
+        if (vv.getClass().equals(NullableVarCharVector.class)) {
+                NullableVarCharVector v = (NullableVarCharVector) vv;
+                prec = v.getField().getWidth();
+                if(prec > 0) {
+                   AllocationHelper.allocate(v, 65536, prec);
+                } else {
+                   AllocationHelper.allocate(v, 65536, 2000);
+                }
+              } else if (vv.getClass().equals(VarCharVector.class)) {
+                VarCharVector v = (VarCharVector) vv;
+                prec = v.getField().getWidth();
+                if(prec > 0) {
+                    AllocationHelper.allocate(v, 65536, prec);
+                 } else {
+                    AllocationHelper.allocate(v, 65536, 2000);
+                 }
+              } else if (vv.getClass().equals(BigIntVector.class)) {
+                BigIntVector v = (BigIntVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(NullableBigIntVector.class)) {
+                NullableBigIntVector v = (NullableBigIntVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(DateVector.class)) {
+                DateVector v = (DateVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(NullableDateVector.class)) {
+                NullableDateVector v = (NullableDateVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(Decimal38DenseVector.class)) {
+                Decimal38DenseVector v = (Decimal38DenseVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(NullableDecimal38DenseVector.class)) {
+                NullableDecimal38DenseVector v = (NullableDecimal38DenseVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(IntVector.class)) {
+                IntVector v = (IntVector) vv;
+                v.allocateNew(65536);
+              } else if (vv.getClass().equals(NullableIntVector.class)) {
+                NullableIntVector v = (NullableIntVector) vv;
+                v.allocateNew(65536);
+              }
+      }
+    } catch (NullPointerException e) {
+      throw new OutOfMemoryException();
+    }
+  }
+
+  @Override
+  public void cleanup() {
+    // TODO Auto-generated method stub
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
new file mode 100644
index 0000000..fae0e81
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcScanSpec.java
@@ -0,0 +1,76 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+public class MPJdbcScanSpec {
+
+  private String database;
+  private String table;
+  private String columns;
+
+  @JsonIgnore
+  private List<String> filters;
+
+  @JsonCreator
+  public MPJdbcScanSpec(@JsonProperty("database") String database,
+      @JsonProperty("table") String table, @JsonProperty("columns") String columns) {
+    this.database = database;
+    this.table = table;
+    this.columns = columns;
+  }
+
+  public MPJdbcScanSpec(String database, String table, List<String> filters, String columns) {
+    this.database = database;
+    this.table = table;
+    this.filters = filters;
+    this.columns = columns;
+  }
+
+  public String getDatabase() {
+    return this.database;
+  }
+
+  public String getTable() {
+    return this.table;
+  }
+
+  public List<String> getFilters() {
+    return this.filters;
+  }
+
+  public String getColumns() {
+    return this.columns;
+  }
+  @Override
+  public String toString() {
+    return "MPJdbcScanSpec [Database=" + database + ", table=" + table
+        + ", columns=" + columns + ", filters=" + filters + "]";
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    // TODO Auto-generated method stub
+    return super.equals(obj);
+  }
+}

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
new file mode 100644
index 0000000..f3169fc
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaConfig.java
@@ -0,0 +1,80 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+
+import com.fasterxml.jackson.annotation.JsonIgnoreProperties;
+import com.fasterxml.jackson.annotation.JsonProperty;
+
+/**
+ * Stores the workspace related config. A workspace has: - location which is a
+ * path. - writable flag to indicate whether the location supports creating new
+ * tables. - default storage format for new tables created in this workspace.
+ */
+
+public class MPJdbcSchemaConfig {
+
+  /** Default workspace is a root directory which supports read, but not write. */
+  public static final MPJdbcSchemaConfig DEFAULT = new MPJdbcSchemaConfig("jdbc://", "",
+      "");
+
+  private final String uri;
+  private final String username;
+  private final String passwd;
+
+  public MPJdbcSchemaConfig(@JsonProperty("uri") String uri,
+      @JsonProperty("username") String username,
+      @JsonProperty("passwd") String passwd) {
+    this.uri = uri;
+    this.username = username;
+    this.passwd = passwd;
+  }
+
+  public String getUri() {
+    return uri;
+  }
+
+  public boolean isWritable() {
+    return false;
+  }
+
+  public String getUsername() {
+    return username;
+  }
+
+  public String getPassword() {
+    return passwd;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == this) {
+      return true;
+    }
+
+    if (obj == null || !(obj instanceof MPJdbcSchemaConfig)) {
+      return false;
+    }
+
+    MPJdbcSchemaConfig that = (MPJdbcSchemaConfig) obj;
+    return ((this.uri == null && that.uri == null) || this.uri.equals(that.uri))
+        && this.passwd == that.passwd
+        && ((this.username == null && that.username == null) || this.username
+            .equals(that.username));
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
new file mode 100644
index 0000000..0e28c8d
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaFilter.java
@@ -0,0 +1,23 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+public class MPJdbcSchemaFilter {
+public MPJdbcSchemaFilter() {
+// TODO Auto-generated constructor stub
+}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/drill/blob/8478e9fb/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
new file mode 100644
index 0000000..8c92533
--- /dev/null
+++ b/contrib/storage-mpjdbc/src/main/java/org/apache/drill/exec/store/mpjdbc/MPJdbcSchemaSubScan.java
@@ -0,0 +1,55 @@
+/**
+ * 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.drill.exec.store.mpjdbc;
+import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.store.mpjdbc.MPJdbcSchemaFilter;
+
+import com.fasterxml.jackson.annotation.JsonCreator;
+import com.fasterxml.jackson.annotation.JsonProperty;
+public class MPJdbcSchemaSubScan extends AbstractSubScan {
+private final String table;
+private final MPJdbcSchemaFilter filter;
+private final String userName;
+
+  @JsonCreator
+  public MPJdbcSchemaSubScan(@JsonProperty("userName") String userName,
+       @JsonProperty("table") String table,
+       @JsonProperty("filter") MPJdbcSchemaFilter filter) {
+    super(userName);
+    this.table = table;
+    this.filter = filter;
+    this.userName = userName;
+  }
+  @JsonProperty("table")
+  public String getTable() {
+    return table;
+  }
+  @JsonProperty("filter")
+  public MPJdbcSchemaFilter getFilter() {
+    return filter;
+  }
+  @JsonProperty("userName")
+  public String getUserName() {
+    return this.userName;
+  }
+@Override
+public int getOperatorType() {
+// TODO Auto-generated method stub
+return 0;
+}
+}
\ No newline at end of file