You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@drill.apache.org by ja...@apache.org on 2014/04/20 03:18:09 UTC

[01/51] [abbrv] Start on move towards optiq for I --> p

Repository: incubator-drill
Updated Branches:
  refs/heads/master 838fda824 -> ecaa838fe


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index bdbb642..ef1529c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -26,14 +26,18 @@ import net.hydromatic.optiq.tools.ValidationException;
 
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.common.BaseScreenRel;
+import org.apache.drill.exec.planner.common.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
-import org.apache.drill.exec.planner.logical.DrillStoreRel;
+import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.store.StoragePluginRegistry.DrillSchemaFactory;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.sql.SqlExplain;
 import org.eigenbase.sql.SqlKind;
 import org.eigenbase.sql.SqlLiteral;
@@ -45,12 +49,25 @@ public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
 
   private final Planner planner;
+  private final static RuleSet[] RULES = new RuleSet[]{DrillRuleSets.DRILL_BASIC_RULES, DrillRuleSets.DRILL_PHYSICAL_MEM};
+  private final static int LOGICAL_RULES = 0;
+  private final static int PHYSICAL_MEM_RULES = 1;
   
   public DrillSqlWorker(DrillSchemaFactory schemaFactory) throws Exception {
     this.planner = Frameworks.getPlanner(ConnectionConfig.Lex.MYSQL, schemaFactory, SqlStdOperatorTable.instance(), new RuleSet[]{DrillRuleSets.DRILL_BASIC_RULES});
   }
   
-  public LogicalPlan getPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
+  private class RelResult{
+    final ResultMode mode;
+    final RelNode node;
+    public RelResult(ResultMode mode, RelNode node) {
+      super();
+      this.mode = mode;
+      this.node = node;
+    }
+  }
+  
+  private RelResult getRel(String sql) throws SqlParseException, ValidationException, RelConversionException{
     SqlNode sqlNode = planner.parse(sql);
 
     ResultMode resultMode = ResultMode.EXEC;
@@ -72,13 +89,20 @@ public class DrillSqlWorker {
     
     SqlNode validatedNode = planner.validate(sqlNode);
     RelNode relNode = planner.convert(validatedNode);
-    RelNode convertedRelNode = planner.transform(0, planner.getEmptyTraitSet().plus(DrillRel.CONVENTION), relNode);
+    return new RelResult(resultMode, relNode);
+  }
+  
+  
+  
+  public LogicalPlan getLogicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
+    RelResult result = getRel(sql);
+    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
     }else{
       convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
     }
-    DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), resultMode);
+    DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), result.mode);
     implementor.go( (DrillRel) convertedRelNode);
     planner.close();
     planner.reset();
@@ -87,4 +111,13 @@ public class DrillSqlWorker {
   }
 
   
+  public PhysicalPlan getPhysicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
+    RelResult result = getRel(sql);
+    RelTraitSet traits = planner.getEmptyTraitSet().plus(Prel.DRILL_PHYSICAL);
+    RelNode transformed = planner.transform(PHYSICAL_MEM_RULES, traits, result.node);
+    planner.close();
+    planner.reset();
+    return null;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
index d4aabb4..470a33d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
@@ -34,15 +34,15 @@ import org.apache.drill.common.logical.data.Project;
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.common.logical.data.Union;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
+import org.apache.drill.exec.planner.common.BaseFilterRel;
+import org.apache.drill.exec.planner.common.DrillLimitRel;
+import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.planner.common.DrillUnionRel;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
-import org.apache.drill.exec.planner.logical.DrillFilterRel;
 import org.apache.drill.exec.planner.logical.DrillJoinRel;
-import org.apache.drill.exec.planner.logical.DrillLimitRel;
-import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
-import org.apache.drill.exec.planner.logical.DrillScanRel;
 import org.apache.drill.exec.planner.logical.DrillSortRel;
-import org.apache.drill.exec.planner.logical.DrillUnionRel;
 import org.apache.drill.exec.planner.logical.ScanFieldDeterminer;
 import org.apache.drill.exec.planner.logical.ScanFieldDeterminer.FieldList;
 import org.eigenbase.rel.InvalidRelException;
@@ -90,7 +90,7 @@ public class ConversionContext implements ToRelContext {
   
   public RelTraitSet getLogicalTraits(){
     RelTraitSet set = RelTraitSet.createEmpty();
-    set.add(DrillRel.CONVENTION);
+    set.add(DrillRel.DRILL_LOGICAL);
     return set;
   }
   
@@ -121,17 +121,17 @@ public class ConversionContext implements ToRelContext {
 
     @Override
     public RelNode visitScan(Scan scan, ConversionContext context){
-      return DrillScanRel.convert(scan, context);
+      return BaseScanRel.convert(scan, context);
     }
 
     @Override
     public RelNode visitFilter(Filter filter, ConversionContext context) throws InvalidRelException{
-      return DrillFilterRel.convert(filter, context);
+      return BaseFilterRel.convert(filter, context);
     }
 
     @Override
     public RelNode visitProject(Project project, ConversionContext context) throws InvalidRelException{
-      return DrillProjectRel.convert(project, context);
+      return BaseProjectRel.convert(project, context);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/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 8baa72a..3081b46 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,7 +21,7 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.JSONOptions;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 
 public abstract class AbstractStoragePlugin implements StoragePlugin{
@@ -46,9 +46,10 @@ public abstract class AbstractStoragePlugin implements StoragePlugin{
   }
 
   @Override
-  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException {
+  public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException {
     throw new UnsupportedOperationException();
   }
-
+  
+  
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/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 2e54b0d..1c986a9 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
@@ -23,7 +23,8 @@ import java.util.List;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
-import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 
 public interface StoragePlugin {
@@ -41,8 +42,9 @@ public interface StoragePlugin {
    * @return
    * @throws IOException
    */
-  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException;
+  public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException;
   
   public Schema createAndAddSchema(SchemaPlus parent);
+  public StoragePluginConfig getConfig();
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index d25d501..120fdf4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -25,9 +25,10 @@ import java.util.Map;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
+import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
-import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
@@ -75,11 +76,11 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
       
       List<WorkspaceSchemaFactory> factories = null;
       if(config.workspaces == null || config.workspaces.isEmpty()){
-        factories = Collections.singletonList(new WorkspaceSchemaFactory("default", name, fs, "/", matchers));
+        factories = Collections.singletonList(new WorkspaceSchemaFactory(this, "default", name, fs, "/", matchers));
       }else{
         factories = Lists.newArrayList();
         for(Map.Entry<String, String> space : config.workspaces.entrySet()){
-          factories.add(new WorkspaceSchemaFactory(space.getKey(), name, fs, space.getValue(), matchers));
+          factories.add(new WorkspaceSchemaFactory(this, space.getKey(), name, fs, space.getValue(), matchers));
         }
       }
       this.schemaFactory = new FileSystemSchemaFactory(name, factories);
@@ -94,8 +95,13 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
   }
   
   @Override
-  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException {
-    FormatSelection formatSelection = scan.getSelection().getWith(context.getConfig(), FormatSelection.class);
+  public StoragePluginConfig getConfig() {
+    return config;
+  }
+
+  @Override
+  public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException {
+    FormatSelection formatSelection = selection.getWith(context.getConfig(), FormatSelection.class);
     FormatPlugin plugin;
     if(formatSelection.getFormat() instanceof NamedFormatPluginConfig){
       plugin = formatsByName.get( ((NamedFormatPluginConfig) formatSelection.getFormat()).name);
@@ -103,7 +109,7 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
       plugin = formatPluginsByConfig.get(formatSelection.getFormat());
     }
     if(plugin == null) throw new IOException(String.format("Failure getting requested format plugin named '%s'.  It was not one of the format plugins registered.", formatSelection.getFormat()));
-    return plugin.getGroupScan(scan.getOutputReference(), formatSelection.getSelection());
+    return plugin.getGroupScan(formatSelection.getSelection());
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
index a37142e..73e414c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FormatPlugin.java
@@ -39,7 +39,7 @@ public interface FormatPlugin {
   
   public FormatMatcher getMatcher();
   
-  public AbstractGroupScan getGroupScan(FieldReference outputRef, FileSelection selection) throws IOException;
+  public AbstractGroupScan getGroupScan(FileSelection selection) throws IOException;
 
   public List<QueryOptimizerRule> getOptimizerRules();
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index c69edb7..2ce7648 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -43,10 +43,12 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
   private final DrillFileSystem fs;
   private final String storageEngineName;
   private final String schemaName;
+  private final FileSystemPlugin plugin;
 
-  public WorkspaceSchemaFactory(String schemaName, String storageEngineName, DrillFileSystem fileSystem, String path,
+  public WorkspaceSchemaFactory(FileSystemPlugin plugin, String schemaName, String storageEngineName, DrillFileSystem fileSystem, String path,
       List<FormatMatcher> formatMatchers) throws ExecutionSetupException {
     this.fs = fileSystem;
+    this.plugin = plugin;
     this.root = new Path(path);
     this.fileMatchers = Lists.newArrayList();
     this.dirMatchers = Lists.newArrayList();
@@ -76,7 +78,7 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
           try {
             Object selection = m.isReadable(fileSelection);
             if (selection != null)
-              return new DynamicDrillTable(storageEngineName, selection, m.getFormatPlugin().getStorageConfig());
+              return new DynamicDrillTable(plugin, storageEngineName, selection, m.getFormatPlugin().getStorageConfig());
           } catch (IOException e) {
             logger.debug("File read failed.", e);
           }
@@ -87,7 +89,7 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
       for (FormatMatcher m : fileMatchers) {
         Object selection = m.isReadable(fileSelection);
         if (selection != null)
-          return new DynamicDrillTable(storageEngineName, selection, m.getFormatPlugin().getStorageConfig());
+          return new DynamicDrillTable(plugin, storageEngineName, selection, m.getFormatPlugin().getStorageConfig());
       }
       return null;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
index d7949c3..780ec14 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyFormatPlugin.java
@@ -92,21 +92,21 @@ public abstract class EasyFormatPlugin<T extends FormatPluginConfig> implements
     return blockSplittable;
   };
 
-  public abstract RecordReader getRecordReader(FragmentContext context, FileWork fileWork, FieldReference ref, List<SchemaPath> columns) throws ExecutionSetupException;
+  public abstract RecordReader getRecordReader(FragmentContext context, FileWork fileWork, List<SchemaPath> columns) throws ExecutionSetupException;
 
   
   RecordBatch getBatch(FragmentContext context, EasySubScan scan) throws ExecutionSetupException {
     List<RecordReader> readers = Lists.newArrayList();
     for(FileWork work : scan.getWorkUnits()){
-      readers.add(getRecordReader(context, work, scan.getRef(), scan.getColumns())); 
+      readers.add(getRecordReader(context, work, scan.getColumns())); 
     }
     
     return new ScanBatch(context, readers.iterator());
   }
   
   @Override
-  public AbstractGroupScan getGroupScan(FieldReference outputRef, FileSelection selection) throws IOException {
-    return new EasyGroupScan(selection, this, outputRef, null);
+  public AbstractGroupScan getGroupScan(FileSelection selection) throws IOException {
+    return new EasyGroupScan(selection, this, null);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
index a7f556e..6015865 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasyGroupScan.java
@@ -55,7 +55,6 @@ public class EasyGroupScan extends AbstractGroupScan{
 
   private final FileSelection selection;
   private final EasyFormatPlugin<?> formatPlugin;
-  private final FieldReference ref;
   private final int maxWidth;
   private final List<SchemaPath> columns;
   
@@ -69,7 +68,6 @@ public class EasyGroupScan extends AbstractGroupScan{
       @JsonProperty("storage") StoragePluginConfig storageConfig, //
       @JsonProperty("format") FormatPluginConfig formatConfig, //
       @JacksonInject StoragePluginRegistry engineRegistry, // 
-      @JsonProperty("ref") FieldReference ref, //
       @JsonProperty("columns") List<SchemaPath> columns
       ) throws IOException, ExecutionSetupException {
     
@@ -77,20 +75,17 @@ public class EasyGroupScan extends AbstractGroupScan{
     Preconditions.checkNotNull(formatPlugin, "Unable to load format plugin for provided format config.");
     this.selection = new FileSelection(files, true);
     this.maxWidth = selection.getFileStatusList(formatPlugin.getFileSystem()).size();
-    this.ref = ref;
     this.columns = columns;
   }
   
   public EasyGroupScan(
       FileSelection selection, //
       EasyFormatPlugin<?> formatPlugin, // 
-      FieldReference ref, //
       List<SchemaPath> columns
       ) throws IOException{
     this.selection = selection;
     this.maxWidth = selection.getFileStatusList(formatPlugin.getFileSystem()).size();
     this.formatPlugin = formatPlugin;
-    this.ref = ref;
     this.columns = columns;
   }
 
@@ -161,7 +156,7 @@ public class EasyGroupScan extends AbstractGroupScan{
     Preconditions.checkArgument(!filesForMinor.isEmpty(),
         String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId));
 
-    return new EasySubScan(convert(filesForMinor), formatPlugin, ref, columns);
+    return new EasySubScan(convert(filesForMinor), formatPlugin, columns);
   }
   
   private List<FileWorkImpl> convert(List<CompleteFileWork> list){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
index 72d1fe6..6631a6a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/easy/EasySubScan.java
@@ -42,7 +42,6 @@ public class EasySubScan extends AbstractSubScan{
 
   private final List<FileWorkImpl> files;
   private final EasyFormatPlugin<?> formatPlugin;
-  private final FieldReference ref;
   private final List<SchemaPath> columns;
   
   @JsonCreator
@@ -51,21 +50,18 @@ public class EasySubScan extends AbstractSubScan{
       @JsonProperty("storage") StoragePluginConfig storageConfig, //
       @JsonProperty("format") FormatPluginConfig formatConfig, //
       @JacksonInject StoragePluginRegistry engineRegistry, // 
-      @JsonProperty("ref") FieldReference ref, //
       @JsonProperty("columns") List<SchemaPath> columns //
       ) throws IOException, ExecutionSetupException {
     
     this.formatPlugin = (EasyFormatPlugin<?>) engineRegistry.getFormatPlugin(storageConfig, formatConfig);
     Preconditions.checkNotNull(this.formatPlugin);
     this.files = files;
-    this.ref = ref;
     this.columns = columns;
   }
   
-  public EasySubScan(List<FileWorkImpl> files, EasyFormatPlugin<?> plugin, FieldReference ref, List<SchemaPath> columns){
+  public EasySubScan(List<FileWorkImpl> files, EasyFormatPlugin<?> plugin, List<SchemaPath> columns){
     this.formatPlugin = plugin;
     this.files = files;
-    this.ref = ref;
     this.columns = columns;
   }
   
@@ -89,11 +85,6 @@ public class EasySubScan extends AbstractSubScan{
     return formatPlugin.getConfig();
   }
   
-  @JsonProperty("ref")
-  public FieldReference getRef() {
-    return ref;
-  }
-  
   @JsonProperty("columns")
   public List<SchemaPath> getColumns(){
     return columns;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
index 82bf3bf..807c67e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/easy/json/JSONFormatPlugin.java
@@ -45,9 +45,9 @@ public class JSONFormatPlugin extends EasyFormatPlugin<JSONFormatConfig> {
   }
   
   @Override
-  public RecordReader getRecordReader(FragmentContext context, FileWork fileWork, FieldReference ref,
+  public RecordReader getRecordReader(FragmentContext context, FileWork fileWork, 
       List<SchemaPath> columns) throws ExecutionSetupException {
-    return new JSONRecordReader(context, fileWork.getPath(), this.getFileSystem().getUnderlying(), ref, columns);
+    return new JSONRecordReader(context, fileWork.getPath(), this.getFileSystem().getUnderlying(), columns);
   }
 
   @JsonTypeName("json")

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/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 d327b77..ac03444 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
@@ -80,23 +80,21 @@ public class JSONRecordReader implements RecordReader {
   private OutputMutator outputMutator;
   private BufferAllocator allocator;
   private int batchSize;
-  private final FieldReference ref;
   private final List<SchemaPath> columns;
 
   public JSONRecordReader(FragmentContext fragmentContext, String inputPath, FileSystem fileSystem, int batchSize,
-                          FieldReference ref, List<SchemaPath> columns) {
+                          List<SchemaPath> columns) {
     this.hadoopPath = new Path(inputPath);
     this.fileSystem = fileSystem;
     this.allocator = fragmentContext.getAllocator();
     this.batchSize = batchSize;
     valueVectorMap = Maps.newHashMap();
-    this.ref = ref;
     this.columns = columns;
   }
 
-  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, FileSystem fileSystem, FieldReference ref,
+  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, FileSystem fileSystem, 
                           List<SchemaPath> columns) {
-    this(fragmentContext, inputPath, fileSystem, DEFAULT_LENGTH, ref, columns);
+    this(fragmentContext, inputPath, fileSystem, DEFAULT_LENGTH, columns);
   }
 
   private JsonParser getParser() {
@@ -149,7 +147,6 @@ public class JSONRecordReader implements RecordReader {
       // Garbage collect fields never referenced in this batch
       for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
         diffSchema.addRemovedField(field);
-        outputMutator.removeField(field.getAsMaterializedField(ref));
       }
 
       if (diffSchema.isChanged()) {
@@ -510,7 +507,7 @@ public class JSONRecordReader implements RecordReader {
   }
 
   private VectorHolder getOrCreateVectorHolder(Field field) throws SchemaChangeException {
-    String fullFieldName = ref != null ? ref.getPath() + "." + field.getFullFieldName() : field.getFullFieldName();
+    String fullFieldName = field.getFullFieldName();
     VectorHolder holder = valueVectorMap.get(fullFieldName);
 
     if (holder == null) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index a1d575c..46d85a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -22,8 +22,8 @@ import java.io.IOException;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
+import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.hive.schema.HiveSchemaFactory;
@@ -37,7 +37,6 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveStoragePlugin.class);
   
   private final HiveStoragePluginConfig config;
-  private final HiveConf hiveConf;
   private final HiveSchemaFactory schemaFactory;
   private final DrillbitContext context;
   private final String name;
@@ -45,8 +44,7 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
   public HiveStoragePlugin(HiveStoragePluginConfig config, DrillbitContext context, String name) throws ExecutionSetupException {
     this.config = config;
     this.context = context;
-    this.schemaFactory = new HiveSchemaFactory(config, name, config.getHiveConf());
-    this.hiveConf = config.getHiveConf();
+    this.schemaFactory = new HiveSchemaFactory(this, name, config.getHiveConf());
     this.name = name;
   }
 
@@ -63,8 +61,8 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public HiveScan getPhysicalScan(Scan scan) throws IOException {
-    HiveReadEntry hiveReadEntry = scan.getSelection().getListWith(new ObjectMapper(), new TypeReference<HiveReadEntry>(){});
+  public HiveScan getPhysicalScan(JSONOptions selection) throws IOException {
+    HiveReadEntry hiveReadEntry = selection.getListWith(new ObjectMapper(), new TypeReference<HiveReadEntry>(){});
     try {
       return new HiveScan(hiveReadEntry, this, null);
     } catch (ExecutionSetupException e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
index abec2c5..7d6bc72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/DrillHiveTable.java
@@ -23,6 +23,7 @@ import java.util.ArrayList;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
+import org.apache.drill.exec.store.hive.HiveStoragePlugin;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.ql.metadata.Table;
 import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector;
@@ -38,8 +39,8 @@ public class DrillHiveTable extends DrillTable{
   
   private final Table hiveTable;
   
-  public DrillHiveTable(String storageEngineName, HiveReadEntry readEntry, StoragePluginConfig storageEngineConfig) {
-    super(storageEngineName, readEntry, storageEngineConfig);
+  public DrillHiveTable(String storageEngineName, HiveStoragePlugin plugin, HiveReadEntry readEntry) {
+    super(storageEngineName, plugin, readEntry);
     this.hiveTable = new org.apache.hadoop.hive.ql.metadata.Table(readEntry.getTable());
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index 091381f..4e80afc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -33,6 +33,7 @@ import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.SchemaHolder;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
+import org.apache.drill.exec.store.hive.HiveStoragePlugin;
 import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
 import org.apache.drill.exec.store.hive.HiveTable;
 import org.apache.hadoop.hive.conf.HiveConf;
@@ -58,12 +59,12 @@ public class HiveSchemaFactory implements SchemaFactory {
   private LoadingCache<String, List<String>> databases;
   private LoadingCache<String, List<String>> tableNameLoader;
   private LoadingCache<String, LoadingCache<String, HiveReadEntry>> tableLoaders;
-  private HiveStoragePluginConfig pluginConfig;
+  private HiveStoragePlugin plugin;
   private final String schemaName;
 
-  public HiveSchemaFactory(HiveStoragePluginConfig pluginConfig, String name, HiveConf hiveConf) throws ExecutionSetupException {
+  public HiveSchemaFactory(HiveStoragePlugin plugin, String name, HiveConf hiveConf) throws ExecutionSetupException {
     this.schemaName = name;
-    this.pluginConfig = pluginConfig;
+    this.plugin = plugin;
     
     try {
       this.mClient = new HiveMetaStoreClient(hiveConf);
@@ -255,7 +256,7 @@ public class HiveSchemaFactory implements SchemaFactory {
     DrillTable getDrillTable(String dbName, String t){
       HiveReadEntry entry = getSelectionBaseOnName(dbName, t);
       if(entry == null) return null;
-      return new DrillHiveTable(schemaName, entry, pluginConfig);
+      return new DrillHiveTable(schemaName, plugin, entry);
     }
     
     HiveReadEntry getSelectionBaseOnName(String dbName, String t) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 0578b06..0d1635f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -23,7 +23,8 @@ import java.util.ArrayList;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
-import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
@@ -35,14 +36,16 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 public class MockStorageEngine extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
 
+  private final MockStorageEngineConfig configuration;
+  
   public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
-
+    this.configuration = configuration;
   }
 
   @Override
-  public AbstractGroupScan getPhysicalScan(Scan scan) throws IOException {
+  public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException {
 
-    ArrayList<MockScanEntry> readEntries = scan.getSelection().getListWith(new ObjectMapper(),
+    ArrayList<MockScanEntry> readEntries = selection.getListWith(new ObjectMapper(),
         new TypeReference<ArrayList<MockScanEntry>>() {
         });
     
@@ -54,5 +57,10 @@ public class MockStorageEngine extends AbstractStoragePlugin {
     return null;
   }
 
+  @Override
+  public StoragePluginConfig getConfig() {
+    return configuration;
+  }
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
index bfaaa45..cde9b08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetFormatPlugin.java
@@ -100,8 +100,8 @@ public class ParquetFormatPlugin implements FormatPlugin{
   }
 
   @Override
-  public ParquetGroupScan getGroupScan(FieldReference outputRef, FileSelection selection) throws IOException {
-    return new ParquetGroupScan( selection.getFileStatusList(fs), this, outputRef);
+  public ParquetGroupScan getGroupScan(FileSelection selection) throws IOException {
+    return new ParquetGroupScan( selection.getFileStatusList(fs), this);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 185fc0d..47218fe 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -87,7 +87,6 @@ public class ParquetGroupScan extends AbstractGroupScan {
   private final ParquetFormatPlugin formatPlugin;
   private final ParquetFormatConfig formatConfig;
   private final FileSystem fs;
-  private final FieldReference ref;
   private List<EndpointAffinity> endpointAffinities;
 
   private List<SchemaPath> columns;
@@ -107,7 +106,6 @@ public class ParquetGroupScan extends AbstractGroupScan {
       @JsonProperty("storage") StoragePluginConfig storageConfig, //
       @JsonProperty("format") FormatPluginConfig formatConfig, //
       @JacksonInject StoragePluginRegistry engineRegistry, // 
-      @JsonProperty("ref") FieldReference ref, //
       @JsonProperty("columns") List<SchemaPath> columns //
       ) throws IOException, ExecutionSetupException {
     engineRegistry.init(DrillConfig.create());
@@ -120,14 +118,12 @@ public class ParquetGroupScan extends AbstractGroupScan {
     this.fs = formatPlugin.getFileSystem().getUnderlying();
     this.formatConfig = formatPlugin.getConfig();
     this.entries = entries;
-    this.ref = ref;
     this.readFooterFromEntries();
 
   }
 
   public ParquetGroupScan(List<FileStatus> files, //
-      ParquetFormatPlugin formatPlugin, //
-      FieldReference ref) //
+      ParquetFormatPlugin formatPlugin) //
       throws IOException {
     this.formatPlugin = formatPlugin;
     this.columns = null;
@@ -139,7 +135,6 @@ public class ParquetGroupScan extends AbstractGroupScan {
       entries.add(new ReadEntryWithPath(file.getPath().toString()));
     }
     
-    this.ref = ref;
     readFooter(files);
   }
 
@@ -282,7 +277,7 @@ public class ParquetGroupScan extends AbstractGroupScan {
     Preconditions.checkArgument(!rowGroupsForMinor.isEmpty(),
         String.format("MinorFragmentId %d has no read entries assigned", minorFragmentId));
 
-    return new ParquetRowGroupScan(formatPlugin, convertToReadEntries(rowGroupsForMinor), ref, columns);
+    return new ParquetRowGroupScan(formatPlugin, convertToReadEntries(rowGroupsForMinor), columns);
   }
 
   
@@ -296,10 +291,7 @@ public class ParquetGroupScan extends AbstractGroupScan {
     }
     return entries;
   }
-  
-  public FieldReference getRef() {
-    return ref;
-  }
+
 
   @Override
   public int getMaxParallelizationWidth() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index 9e1cc66..a3c57cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -72,7 +72,6 @@ class ParquetRecordReader implements RecordReader {
   private int bitWidthAllFixedFields;
   private boolean allFieldsFixedLength;
   private int recordsPerBatch;
-  private final FieldReference ref;
   private long totalRecords;
   private long rowGroupOffset;
 
@@ -93,22 +92,24 @@ class ParquetRecordReader implements RecordReader {
 
   int rowGroupIndex;
 
-  public ParquetRecordReader(FragmentContext fragmentContext,
-                             String path, int rowGroupIndex, FileSystem fs,
-                             CodecFactoryExposer codecFactoryExposer, ParquetMetadata footer, FieldReference ref,
+  public ParquetRecordReader(FragmentContext fragmentContext, //
+                             String path, //
+                             int rowGroupIndex, //
+                             FileSystem fs, //
+                             CodecFactoryExposer codecFactoryExposer, //
+                             ParquetMetadata footer, //
                              List<SchemaPath> columns) throws ExecutionSetupException {
-    this(fragmentContext, DEFAULT_BATCH_LENGTH_IN_BITS, path, rowGroupIndex, fs, codecFactoryExposer, footer, ref,
+    this(fragmentContext, DEFAULT_BATCH_LENGTH_IN_BITS, path, rowGroupIndex, fs, codecFactoryExposer, footer,
         columns);
   }
 
   public ParquetRecordReader(FragmentContext fragmentContext, long batchSize,
                              String path, int rowGroupIndex, FileSystem fs,
-                             CodecFactoryExposer codecFactoryExposer, ParquetMetadata footer, FieldReference ref,
+                             CodecFactoryExposer codecFactoryExposer, ParquetMetadata footer,
                              List<SchemaPath> columns) throws ExecutionSetupException {
     this.allocator = fragmentContext.getAllocator();
     hadoopPath = new Path(path);
     fileSystem = fs;
-    this.ref = ref;
     this.codecFactoryExposer = codecFactoryExposer;
     this.rowGroupIndex = rowGroupIndex;
     this.batchSize = batchSize;
@@ -255,11 +256,7 @@ class ParquetRecordReader implements RecordReader {
   }
 
   private SchemaPath toFieldName(String[] paths) {
-    if(this.ref == null){
-      return new SchemaPath(Joiner.on('/').join(paths), ExpressionPosition.UNKNOWN);
-    }else{
-      return ref.getChild(paths);
-    }
+    return new SchemaPath(Joiner.on('/').join(paths), ExpressionPosition.UNKNOWN);
   }
 
   private TypeProtos.DataMode getDataMode(ColumnDescriptor column) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
index 0e672d0..0b1a788 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRowGroupScan.java
@@ -52,7 +52,6 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
   public final ParquetFormatConfig formatConfig;
   private final ParquetFormatPlugin formatPlugin;
   private final List<RowGroupReadEntry> rowGroupReadEntries;
-  private final FieldReference ref;
   private final List<SchemaPath> columns;
 
   @JsonCreator
@@ -61,7 +60,6 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
       @JsonProperty("storage") StoragePluginConfig storageConfig, //
       @JsonProperty("format") FormatPluginConfig formatConfig, //
       @JsonProperty("entries") LinkedList<RowGroupReadEntry> rowGroupReadEntries, //
-      @JsonProperty("ref") FieldReference ref, //
       @JsonProperty("columns") List<SchemaPath> columns //
   ) throws ExecutionSetupException {
 
@@ -72,19 +70,16 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     Preconditions.checkNotNull(formatPlugin);
     this.rowGroupReadEntries = rowGroupReadEntries;
     this.formatConfig = formatPlugin.getConfig();
-    this.ref = ref;
     this.columns = columns;
   }
 
   public ParquetRowGroupScan( //
       ParquetFormatPlugin formatPlugin, //
       List<RowGroupReadEntry> rowGroupReadEntries, //
-      FieldReference ref, //
       List<SchemaPath> columns) {
     this.formatPlugin = formatPlugin;
     this.formatConfig = formatPlugin.getConfig();
     this.rowGroupReadEntries = rowGroupReadEntries;
-    this.ref = ref;
     this.columns = columns;
   }
 
@@ -103,10 +98,6 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
     return null;
   }
 
-  public FieldReference getRef() {
-    return ref;
-  }
-
   @Override
   public Size getSize() {
     return null;
@@ -130,7 +121,7 @@ public class ParquetRowGroupScan extends AbstractBase implements SubScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
     Preconditions.checkArgument(children.isEmpty());
-    return new ParquetRowGroupScan(formatPlugin, rowGroupReadEntries, ref, columns);
+    return new ParquetRowGroupScan(formatPlugin, rowGroupReadEntries, columns);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
index 17e7da2..d36dbc0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetScanBatchCreator.java
@@ -78,7 +78,6 @@ public class ParquetScanBatchCreator implements BatchCreator<ParquetRowGroupScan
                 context, e.getPath(), e.getRowGroupIndex(), fs,
                 rowGroupScan.getStorageEngine().getCodecFactoryExposer(),
                 footers.get(e.getPath()),
-                rowGroupScan.getRef(),
                 rowGroupScan.getColumns()
             )
         );

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 08cb599..90b2a4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -346,23 +346,6 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
 
   private void runSQL(String sql) {
     try{
-      DrillSqlWorker sqlWorker = new DrillSqlWorker(context.getFactory());
-      LogicalPlan plan = sqlWorker.getPlan(sql);
-      
-
-      if(plan.getProperties().resultMode == ResultMode.LOGICAL){
-        returnLogical(plan);
-        return;
-      }
-
-      PhysicalPlan physical = convert(plan);
-      
-      if(plan.getProperties().resultMode == ResultMode.PHYSICAL){
-        returnPhysical(physical);
-        return;
-      }
-      
-      runPhysicalPlan(physical);
     }catch(Exception e){
       fail("Failure while parsing sql.", e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
new file mode 100644
index 0000000..68d1eef
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
@@ -0,0 +1,39 @@
+package org.apache.drill.exec.planner.physical;
+
+import mockit.NonStrictExpectations;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.FunctionRegistry;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.planner.sql.DrillSqlWorker;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.junit.Test;
+
+import com.codahale.metrics.MetricRegistry;
+
+public class TestPhysicalPlanning {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestPhysicalPlanning.class);
+  
+  @Test
+  public void testSimpleQuery(final DrillbitContext bitContext) throws Exception{
+    
+    final DrillConfig c = DrillConfig.create();
+    new NonStrictExpectations() {
+      {
+        bitContext.getMetrics();
+        result = new MetricRegistry();
+        bitContext.getAllocator();
+        result = new TopLevelAllocator();
+        bitContext.getConfig();
+        result = c;
+      }
+    };
+    
+    FunctionRegistry reg = new FunctionRegistry(c);
+    StoragePluginRegistry registry = new StoragePluginRegistry(bitContext);
+    DrillSqlWorker worker = new DrillSqlWorker(registry.getSchemaFactory(), reg);
+    worker.getPhysicalPlan("select * from cp.`employee.json`");
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
index e812997..b5ad235 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
@@ -139,7 +139,7 @@ public class JSONRecordReaderTest {
     };
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_1.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
@@ -169,7 +169,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_2.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
@@ -211,7 +211,7 @@ public class JSONRecordReaderTest {
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_2.json").toURI().toString(),
         FileSystem.getLocal(new Configuration()),
-        64, null, Arrays.asList(new SchemaPath("test", ExpressionPosition.UNKNOWN))); // batch only fits 1 int
+        64, Arrays.asList(new SchemaPath("test", ExpressionPosition.UNKNOWN))); // batch only fits 1 int
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
     List<MaterializedField> removedFields = mutator.getRemovedFields();
@@ -246,7 +246,7 @@ public class JSONRecordReaderTest {
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_2.json").toURI().toString(),
         FileSystem.getLocal(new Configuration()),
-        64, null, null); // batch only fits 1 int
+        64, null); // batch only fits 1 int
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
     List<MaterializedField> removedFields = mutator.getRemovedFields();
@@ -304,7 +304,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_3.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
@@ -333,7 +333,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_4.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();
@@ -366,7 +366,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_5.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/test/resources/storage-engines.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/storage-engines.json b/exec/java-exec/src/test/resources/storage-engines.json
index 6e4d23e..73899ee 100644
--- a/exec/java-exec/src/test/resources/storage-engines.json
+++ b/exec/java-exec/src/test/resources/storage-engines.json
@@ -3,10 +3,11 @@
     dfs: {
       type: "file",
       connection: "file:///"
-    },
+    },  
     cp: {
       type: "file",
       connection: "classpath:///"
-    }
-  }  
+    }      
+
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java b/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
index 8cd682b..1e4edca 100644
--- a/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
+++ b/sqlparser/src/test/java/org/apache/drill/jdbc/test/JdbcTest.java
@@ -184,7 +184,6 @@ public class JdbcTest {
 //    Assert.assertTrue(config != null && config instanceof QueueRSE.QueueRSEConfig);
     Scan scan = findOnlyOperator(plan, Scan.class);
     Assert.assertEquals("donuts-json", scan.getStorageEngine());
-    Assert.assertEquals("_MAP", scan.getOutputReference().getPath());
     Project project = findOnlyOperator(plan, Project.class);
     Assert.assertEquals(1, project.getSelections().length);
     Assert.assertEquals(Scan.class, project.getInput().getClass());
@@ -240,7 +239,6 @@ public class JdbcTest {
 //    Assert.assertTrue(config != null && config instanceof QueueRSE.QueueRSEConfig);
     Scan scan = findOnlyOperator(plan, Scan.class);
     Assert.assertEquals("donuts-json", scan.getStorageEngine());
-    Assert.assertEquals("_MAP", scan.getOutputReference().getPath());
     Filter filter = findOnlyOperator(plan, Filter.class);
     Assert.assertTrue(filter.getInput() instanceof Scan);
     Project[] projects = Iterables.toArray(findOperator(plan, Project.class), Project.class);


[38/51] [abbrv] git commit: Remove RemoveSortRule and PushSortPastProjectRule during Drill Logical planning phase.

Posted by ja...@apache.org.
Remove RemoveSortRule and PushSortPastProjectRule during Drill Logical planning phase.


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

Branch: refs/heads/master
Commit: 0b1df5dfad13ab82ca95287eec2bf1ed960aee87
Parents: ef8b59a
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Thu Apr 3 16:51:01 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/planner/logical/DrillRuleSets.java     | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/0b1df5df/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 d17fce3..2a1d3ff 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
@@ -69,7 +69,7 @@ public class DrillRuleSets {
 //      UnionToDistinctRule.INSTANCE,
 //      RemoveTrivialProjectRule.INSTANCE,
 //      RemoveTrivialCalcRule.INSTANCE,
-      RemoveSortRule.INSTANCE,
+//      RemoveSortRule.INSTANCE,
 
 //      TableAccessRule.INSTANCE, //
       //MergeProjectRule.INSTANCE, //
@@ -79,7 +79,7 @@ public class DrillRuleSets {
 //      SwapJoinRule.INSTANCE, //
 //      PushJoinThroughJoinRule.RIGHT, //
 //      PushJoinThroughJoinRule.LEFT, //
-      PushSortPastProjectRule.INSTANCE, //      
+//      PushSortPastProjectRule.INSTANCE, //      
 
       ////////////////////////////////
       DrillScanRule.INSTANCE,


[47/51] [abbrv] git commit: Add support for non-equijoins in the presence of other equijoins between the same tables.

Posted by ja...@apache.org.
Add support for non-equijoins in the presence of other equijoins between the same tables.


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

Branch: refs/heads/master
Commit: 63287658c0628ecfbc56d0d33618ad2219fef438
Parents: 4cfdb3b
Author: Aman Sinha <as...@maprtech.com>
Authored: Fri Apr 4 10:08:24 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/common/DrillJoinRelBase.java   |  4 +-
 .../exec/planner/logical/DrillJoinRel.java      | 27 ++++++++-
 .../exec/planner/logical/DrillJoinRule.java     | 63 +++++++++++++++++++-
 .../exec/planner/logical/DrillRuleSets.java     |  4 +-
 .../exec/planner/physical/MergeJoinPrel.java    |  2 +-
 5 files changed, 91 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63287658/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
index e88faf9..b9c112d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
@@ -45,8 +45,8 @@ import com.google.common.collect.Lists;
  * Base class for logical and physical Joins implemented in Drill.
  */
 public abstract class DrillJoinRelBase extends JoinRelBase implements DrillRelNode {
-  protected final List<Integer> leftKeys = new ArrayList<>();
-  protected final List<Integer> rightKeys = new ArrayList<>();
+  protected List<Integer> leftKeys = Lists.newArrayList();
+  protected List<Integer> rightKeys = Lists.newArrayList() ;
 
   public DrillJoinRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType) throws InvalidRelException {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63287658/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 31a8e6e..a3a4ac9 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
@@ -42,20 +42,41 @@ import org.eigenbase.rex.RexUtil;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.util.Pair;
 
+import com.google.hive12.common.collect.Lists;
+
 /**
  * Join implemented in Drill.
  */
 public class DrillJoinRel extends DrillJoinRelBase implements DrillRel {
+
   /** Creates a DrillJoinRel. */
   public DrillJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType) throws InvalidRelException {
     super(cluster, traits, left, right, condition, joinType);
 
     RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
-    if (!remaining.isAlwaysTrue()) {
+    if (!remaining.isAlwaysTrue() && (leftKeys.size() == 0 || rightKeys.size() == 0)) {
       throw new InvalidRelException("DrillJoinRel only supports equi-join");
     }
   }
+
+  public DrillJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
+      JoinRelType joinType, List<Integer> leftKeys, List<Integer> rightKeys, boolean checkCartesian) throws InvalidRelException {
+    super(cluster, traits, left, right, condition, joinType);
+    
+    assert (leftKeys != null && rightKeys != null);
+    
+    if (checkCartesian)  {
+      List<Integer> tmpLeftKeys = Lists.newArrayList();
+      List<Integer> tmpRightKeys = Lists.newArrayList();
+      RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, tmpLeftKeys, tmpRightKeys);
+      if (!remaining.isAlwaysTrue() && (tmpLeftKeys.size() == 0 || tmpRightKeys.size() == 0)) {
+        throw new InvalidRelException("DrillJoinRel only supports equi-join");
+      }
+    }
+    this.leftKeys = leftKeys;
+    this.rightKeys = rightKeys;
+  }
   
   @Override
   public DrillJoinRel copy(RelTraitSet traitSet, RexNode condition, RelNode left, RelNode right, JoinRelType joinType) {
@@ -141,7 +162,9 @@ public class DrillJoinRel extends DrillJoinRelBase implements DrillRel {
                 );
     }
     RexNode rexCondition = RexUtil.composeConjunction(context.getRexBuilder(), joinConditions, false);
-    return new DrillJoinRel(context.getCluster(), context.getLogicalTraits(), left, right, rexCondition, join.getJoinType());
+    DrillJoinRel joinRel = new DrillJoinRel(context.getCluster(), context.getLogicalTraits(), left, right, rexCondition, join.getJoinType());
+
+    return joinRel;
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63287658/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index 2a895a5..60c4ae0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
@@ -21,8 +21,17 @@ import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.JoinRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.*;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.rex.RexBuilder;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.rex.RexUtil;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.trace.EigenbaseTrace;
 
+import com.beust.jcommander.internal.Lists;
+
+import java.util.ArrayList;
+import java.util.List;
 import java.util.logging.Logger;
 
 /**
@@ -47,9 +56,59 @@ public class DrillJoinRule extends RelOptRule {
 
     final RelNode convertedLeft = convert(left, traits);
     final RelNode convertedRight = convert(right, traits);
+
+    List<Integer> leftKeys = Lists.newArrayList();
+    List<Integer> rightKeys = Lists.newArrayList();
+    int numLeftFields = convertedLeft.getRowType().getFieldCount();
+
+    boolean addFilter = false;
+    RexNode origJoinCondition = join.getCondition();
+    RexNode newJoinCondition = origJoinCondition;
+    
+    RexNode remaining = RelOptUtil.splitJoinCondition(convertedLeft, convertedRight, origJoinCondition, leftKeys, rightKeys);
+    boolean hasEquijoins = (leftKeys.size() == rightKeys.size() && leftKeys.size() > 0) ? true : false;
+
+    // If the join involves equijoins and non-equijoins, then we can process the non-equijoins through  
+    // a filter right after the join
+    if (! remaining.isAlwaysTrue()) {
+      if (hasEquijoins) {
+        addFilter = true;
+        List<RexNode> equijoinList = Lists.newArrayList();
+        List<RelDataTypeField> leftTypes = convertedLeft.getRowType().getFieldList();
+        List<RelDataTypeField> rightTypes = convertedRight.getRowType().getFieldList();
+        RexBuilder builder = join.getCluster().getRexBuilder();
+      
+        for (int i=0; i < leftKeys.size(); i++) {
+          int leftKeyOrdinal = leftKeys.get(i).intValue();
+          int rightKeyOrdinal = rightKeys.get(i).intValue();
+   
+          equijoinList.add(builder.makeCall( 
+              SqlStdOperatorTable.EQUALS,
+              builder.makeInputRef(leftTypes.get(leftKeyOrdinal).getType(), leftKeyOrdinal), 
+              builder.makeInputRef(rightTypes.get(rightKeyOrdinal).getType(), rightKeyOrdinal + numLeftFields)
+             ) );            
+        }
+        newJoinCondition = RexUtil.composeConjunction(builder, equijoinList, false);    
+      } else {
+        tracer.warning("Non-equijoins are only supported in the presence of an equijoin.");
+        return;
+      }
+    }
+    //else {
+    //  
+    //  return;
+    // }
+
     try {
-      call.transformTo(new DrillJoinRel(join.getCluster(), traits, convertedLeft, convertedRight, join.getCondition(),
-          join.getJoinType()));
+      if (!addFilter) {
+       RelNode joinRel = new DrillJoinRel(join.getCluster(), traits, convertedLeft, convertedRight, origJoinCondition, 
+                                         join.getJoinType(), leftKeys, rightKeys, false);
+       call.transformTo(joinRel);
+      } else {
+        RelNode joinRel = new DrillJoinRel(join.getCluster(), traits, convertedLeft, convertedRight, newJoinCondition, 
+                                           join.getJoinType(), leftKeys, rightKeys, false);
+        call.transformTo(new DrillFilterRel(join.getCluster(), traits, joinRel, remaining));
+      } 
     } catch (InvalidRelException e) {
       tracer.warning(e.toString());
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63287658/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 2a1d3ff..e249af3 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
@@ -90,8 +90,8 @@ public class DrillRuleSets {
       DrillLimitRule.INSTANCE,
       DrillSortRule.INSTANCE,
       DrillJoinRule.INSTANCE,
-      DrillUnionRule.INSTANCE      
-
+      DrillUnionRule.INSTANCE,      
+      MergeProjectRule.INSTANCE
       ));
   
   public static final RuleSet DRILL_PHYSICAL_MEM = new DrillRuleSet(ImmutableSet.of( //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63287658/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index bfb2192..0ced205 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -55,7 +55,7 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     }
     
     RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
-    if (!remaining.isAlwaysTrue()) {
+    if (!remaining.isAlwaysTrue() && (leftKeys.size() == 0 || rightKeys.size() == 0)) {
       throw new InvalidRelException("MergeJoinPrel only supports equi-join");
     }
   }


[48/51] [abbrv] git commit: Specify target traits based on converted RelNode's trait, in stead of empty trait sets, when call planner to get the logical plan.

Posted by ja...@apache.org.
Specify target traits based on converted RelNode's trait, in stead of empty trait sets, when call planner to get the logical plan.

Add back RemoveSortRule to Logical rulesets.


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

Branch: refs/heads/master
Commit: 56411a570a3516e4e10942c63f6cf00866e0ab05
Parents: 6328765
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Fri Apr 4 11:56:44 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/planner/logical/DrillRuleSets.java     | 2 +-
 .../java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java   | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/56411a57/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 e249af3..5bd8581 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
@@ -69,7 +69,7 @@ public class DrillRuleSets {
 //      UnionToDistinctRule.INSTANCE,
 //      RemoveTrivialProjectRule.INSTANCE,
 //      RemoveTrivialCalcRule.INSTANCE,
-//      RemoveSortRule.INSTANCE,
+      RemoveSortRule.INSTANCE,
 
 //      TableAccessRule.INSTANCE, //
       //MergeProjectRule.INSTANCE, //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/56411a57/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index c540066..e796f12 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -118,7 +118,7 @@ public class DrillSqlWorker {
     
     System.out.println(RelOptUtil.toString(relNode, SqlExplainLevel.ALL_ATTRIBUTES));
     
-    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
+    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, relNode.getTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
     }else{
@@ -135,7 +135,7 @@ public class DrillSqlWorker {
   public LogicalPlan getLogicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
     RelResult result = getRel(sql);
 
-    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
+    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, result.node.getTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
     }else{


[29/51] [abbrv] git commit: Revert "Create CastExpression and traverse visitor pattern while performing implicit cast in ExpressionTreeMaterializer."

Posted by ja...@apache.org.
Revert "Create CastExpression and traverse visitor pattern while performing implicit cast in ExpressionTreeMaterializer."

This reverts commit b7c76c218a7359e99140bc8ee8eeef65f950d7ee.


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

Branch: refs/heads/master
Commit: eeb853573fbb908f765c6718b193b96d6e829531
Parents: 342f3fd
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Tue Apr 1 15:08:16 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../exec/expr/ExpressionTreeMaterializer.java   | 23 +++++++++++++-------
 1 file changed, 15 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/eeb85357/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 97b965d..d65ff78 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -127,27 +127,34 @@ public class ExpressionTreeMaterializer {
       if (matchedFuncHolder!=null) {
         //Compare parm type against arg type. Insert cast on top of arg, whenever necessary.
         for (int i = 0; i < call.args.size(); ++i) {
-          LogicalExpression currentArg = call.args.get(i);
-
           TypeProtos.MajorType parmType = matchedFuncHolder.getParmMajorType(i);
 
           //Case 1: If  1) the argument is NullExpression
           //            2) the parameter of matchedFuncHolder allows null input, or func's null_handling is NULL_IF_NULL (means null and non-null are exchangable).
           //        then replace NullExpression with a TypedNullConstant
-          if (currentArg.equals(NullExpression.INSTANCE) &&
+          if (call.args.get(i).equals(NullExpression.INSTANCE) &&
             ( parmType.getMode().equals(TypeProtos.DataMode.OPTIONAL) ||
               matchedFuncHolder.getNullHandling() == FunctionTemplate.NullHandling.NULL_IF_NULL)) {
             argsWithCast.add(new TypedNullConstant(parmType));
-          } else if (Types.softEquals(parmType, currentArg.getMajorType(), matchedFuncHolder.getNullHandling() ==
+          } else if (Types.softEquals(parmType, call.args.get(i).getMajorType(), matchedFuncHolder.getNullHandling() ==
             FunctionTemplate.NullHandling.NULL_IF_NULL)) {
             //Case 2: argument and parameter matches. Do nothing.
             argsWithCast.add(call.args.get(i));
           } else {
             //Case 3: insert cast if param type is different from arg type.
-
-            // Create the desired output type and CasExpression and traverse the visitor pattern
-            CastExpression castExpression = new CastExpression(currentArg, parmType, ExpressionPosition.UNKNOWN);
-            argsWithCast.add(castExpression.accept(this, registry));
+            String castFuncName = CastFunctions.getCastFunc(parmType.getMinorType());
+            List<LogicalExpression> castArgs = Lists.newArrayList();
+            castArgs.add(call.args.get(i));  //input_expr
+            FunctionCall castCall = new FunctionCall(castFuncName, castArgs, ExpressionPosition.UNKNOWN);
+            DrillFuncHolder matchedCastFuncHolder = resolver.getBestMatch(
+              registry.getDrillRegistry().getMethods().get(castFuncName), castCall);
+
+            if (matchedCastFuncHolder == null) {
+              logFunctionResolutionError(errorCollector, castCall);
+              return NullExpression.INSTANCE;
+            }
+
+            argsWithCast.add(new DrillFuncHolderExpr(call.getName(), matchedCastFuncHolder, castArgs, ExpressionPosition.UNKNOWN));
           }
         }
         return new DrillFuncHolderExpr(call.getName(), matchedFuncHolder, argsWithCast, call.getPosition());


[34/51] [abbrv] git commit: Add extract functions

Posted by ja...@apache.org.
Add extract functions


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

Branch: refs/heads/master
Commit: dcc102a2a4303d64e919b24d8990c2e115492402
Parents: 3efba1e
Author: vkorukanti <ve...@gmail.com>
Authored: Wed Apr 2 18:48:27 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/src/main/codegen/config.fmpp     |   1 +
 .../src/main/codegen/data/ExtractTypes.tdd      |  20 +++
 .../src/main/codegen/templates/Extract.java     | 118 ++++++++++++++++
 .../drill/exec/expr/EvaluationVisitor.java      |  16 +--
 .../exec/expr/ExpressionTreeMaterializer.java   | 113 ++++++++-------
 .../physical/impl/TestExtractFunctions.java     | 136 +++++++++++++++++++
 .../test/resources/functions/extractFrom.json   |  46 +++++++
 .../src/test/resources/test_simple_time.json    |   3 +
 8 files changed, 381 insertions(+), 72 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/main/codegen/config.fmpp
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/config.fmpp b/exec/java-exec/src/main/codegen/config.fmpp
index 3f19de0..ef82bf2 100644
--- a/exec/java-exec/src/main/codegen/config.fmpp
+++ b/exec/java-exec/src/main/codegen/config.fmpp
@@ -24,6 +24,7 @@ data: {
     aggrtypes1: tdd(../data/AggrTypes1.tdd),
     aggrtypes2: tdd(../data/AggrTypes2.tdd),
     date: tdd(../data/DateTypes.tdd)
+    extract: tdd(../data/ExtractTypes.tdd)
 }
 freemarkerLinks: {
     includes: includes/

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/main/codegen/data/ExtractTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/ExtractTypes.tdd b/exec/java-exec/src/main/codegen/data/ExtractTypes.tdd
new file mode 100644
index 0000000..70d7861
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/data/ExtractTypes.tdd
@@ -0,0 +1,20 @@
+# 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.
+
+{
+  toTypes: [Second, Minute, Hour, Day, Month, Year],
+  fromTypes: [Date, Time, TimeStamp, Interval, IntervalDay, IntervalYear]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/main/codegen/templates/Extract.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Extract.java b/exec/java-exec/src/main/codegen/templates/Extract.java
new file mode 100644
index 0000000..d0e0afe
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/Extract.java
@@ -0,0 +1,118 @@
+/**
+ * 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.
+ */
+<@pp.dropOutputFile />
+<#assign className="GExtract" />
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${className}.java" />
+
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class ${className} {
+
+<#list extract.toTypes as toUnit>
+<#list extract.fromTypes as fromUnit>
+<#if fromUnit == "Date" || fromUnit == "Time" || fromUnit == "TimeStamp">
+  @FunctionTemplate(name = "extract${toUnit}", scope = FunctionTemplate.FunctionScope.SIMPLE,
+      nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class ${toUnit}From${fromUnit} implements DrillSimpleFunc {
+
+    @Param ${fromUnit}Holder in;
+    @Output BigIntHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      org.joda.time.MutableDateTime temp = new org.joda.time.MutableDateTime(in.value, org.joda.time.DateTimeZone.UTC);
+    <#if toUnit == "Second">
+      out.value = temp.getSecondOfMinute();
+    <#elseif toUnit = "Minute">
+      out.value = temp.getMinuteOfHour();
+    <#elseif toUnit = "Hour">
+      out.value = temp.getHourOfDay();
+    <#elseif toUnit = "Day">
+      out.value = temp.getDayOfMonth();
+    <#elseif toUnit = "Month">
+      out.value = temp.getMonthOfYear();
+    <#elseif toUnit = "Year">
+      out.value = temp.getYear();
+    </#if>
+    }
+  }
+<#else>
+  @FunctionTemplate(name = "extract${toUnit}", scope = FunctionTemplate.FunctionScope.SIMPLE,
+      nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class ${toUnit}From${fromUnit} implements DrillSimpleFunc {
+
+    @Param ${fromUnit}Holder in;
+    @Output BigIntHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+    <#if fromUnit == "Interval">
+
+      int years  = (in.months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      int months = (in.months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+
+      int millis = in.milliSeconds;
+
+      int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+
+      int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+
+      int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      org.joda.time.Period temp = new org.joda.time.Period(years, months, 0, in.days, hours, minutes, seconds, millis);
+
+    <#elseif fromUnit == "IntervalDay">
+
+      int millis = in.milliSeconds;
+
+      int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+
+      int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+
+      int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+      org.joda.time.Period temp = new org.joda.time.Period(0, 0, 0, in.days, hours, minutes, seconds, millis);
+
+    <#else>
+
+      int years  = (in.value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      int months = (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+      org.joda.time.Period temp = new org.joda.time.Period(years, months, 0, 0, 0, 0, 0, 0);
+
+    </#if>
+      out.value = temp.get${toUnit}s();
+    }
+  }
+  </#if>
+</#list>
+</#list>
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index b7670ee..aff47db 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -377,19 +377,9 @@ public class EvaluationVisitor {
 
     @Override
     public HoldingContainer visitCastExpression(CastExpression e, ClassGenerator<?> value) throws RuntimeException {
-      // we create
-      MajorType type = e.getMajorType();
-      String castFuncWithType = "cast" + type.getMinorType().name();
-
-      List<LogicalExpression> newArgs = Lists.newArrayList();
-      newArgs.add(e.getInput());  //input_expr
-
-      //VarLen type
-      if (!Types.isFixedWidthType(type)) {
-        newArgs.add(new ValueExpressions.LongExpression(type.getWidth(), null));
-      }
-      FunctionCall fc = new FunctionCall(castFuncWithType, newArgs, e.getPosition());
-      return fc.accept(this, value);    }
+      throw new UnsupportedOperationException("CastExpression is not expected here. "+
+        "It should have been converted to FunctionHolderExpression in materialization");
+    }
   }
 
   private class ConstantFilter extends EvalVisitor {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index d65ff78..1d8070c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -289,70 +289,65 @@ public class ExpressionTreeMaterializer {
       if(castEqual(e.getPosition(), newMajor, input.getMajorType())) return input; // don't do pointless cast.
       
       
-      if(newMinor == MinorType.LATE || newMinor == MinorType.NULL){
-        // if the type still isn't fully bound, leave as cast expression.
-        return new CastExpression(input, e.getMajorType(), e.getPosition());
-      }else{
-        // if the type is fully bound, convert to functioncall and materialze the function.
-        MajorType type = e.getMajorType();
-        String castFuncWithType = "cast" + type.getMinorType().name();
-
-        List<LogicalExpression> newArgs = Lists.newArrayList();
-        newArgs.add(e.getInput());  //input_expr
-
-        //VarLen type
-        if (!Types.isFixedWidthType(type)) {
-          newArgs.add(new ValueExpressions.LongExpression(type.getWidth(), null));
-        }
-        FunctionCall fc = new FunctionCall(castFuncWithType, newArgs, e.getPosition());
-        return fc.accept(this, value);   
+      if(newMinor == MinorType.LATE){
+        throw new UnsupportedOperationException("LATE binding is not supported");
+      } else if (newMinor == MinorType.NULL){
+        // convert it into null expression
+        return NullExpression.INSTANCE;
       }
-      
-      
-      
+
+      // if the type is fully bound, convert to functioncall and materialze the function.
+      MajorType type = e.getMajorType();
+      String castFuncWithType = "cast" + type.getMinorType().name();
+
+      List<LogicalExpression> newArgs = Lists.newArrayList();
+      newArgs.add(e.getInput());  //input_expr
+
+      //VarLen type
+      if (!Types.isFixedWidthType(type)) {
+        newArgs.add(new ValueExpressions.LongExpression(type.getWidth(), null));
+      }
+      FunctionCall fc = new FunctionCall(castFuncWithType, newArgs, e.getPosition());
+      return fc.accept(this, value);
     }
   
-  private boolean castEqual(ExpressionPosition pos, MajorType from, MajorType to){
-    if(!from.getMinorType().equals(to.getMinorType())) return false;
-    switch(from.getMinorType()){
-    case FLOAT4:
-    case FLOAT8:
-    case INT:
-    case BIGINT:
-    case BIT:
-    case TINYINT:
-    case UINT1:
-    case UINT2:
-    case UINT4:
-    case UINT8:      
-      // nothing else matters.
-      return true;
-     
-    case FIXED16CHAR:
-    case FIXEDBINARY:
-    case FIXEDCHAR:
-      // width always matters
-      this.errorCollector.addGeneralError(pos, "Casting fixed width types are not yet supported..");
-      return false;
-      
-    case VAR16CHAR:
-    case VARBINARY:
-    case VARCHAR:
-      if(to.getWidth() < from.getWidth() && to.getWidth() > 0){
-        this.errorCollector.addGeneralError(pos, "Casting from a longer variable length type to a shorter variable length type is not currently supported.");
-        return false;
-      }else{
+    private boolean castEqual(ExpressionPosition pos, MajorType from, MajorType to){
+      if(!from.getMinorType().equals(to.getMinorType())) return false;
+      switch(from.getMinorType()){
+      case FLOAT4:
+      case FLOAT8:
+      case INT:
+      case BIGINT:
+      case BIT:
+      case TINYINT:
+      case UINT1:
+      case UINT2:
+      case UINT4:
+      case UINT8:
+        // nothing else matters.
         return true;
-      }
 
-    default:
-      errorCollector.addGeneralError(pos, String.format("Casting rules are unknown for type %s.", from));
-      return false;
-    
-    }
+      case FIXED16CHAR:
+      case FIXEDBINARY:
+      case FIXEDCHAR:
+        // width always matters
+        this.errorCollector.addGeneralError(pos, "Casting fixed width types are not yet supported..");
+        return false;
 
-  }
-  
-  }
+      case VAR16CHAR:
+      case VARBINARY:
+      case VARCHAR:
+        if(to.getWidth() < from.getWidth() && to.getWidth() > 0){
+          this.errorCollector.addGeneralError(pos, "Casting from a longer variable length type to a shorter variable length type is not currently supported.");
+          return false;
+        }else{
+          return true;
+        }
 
+      default:
+        errorCollector.addGeneralError(pos, String.format("Casting rules are unknown for type %s.", from));
+        return false;
+      }
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
new file mode 100644
index 0000000..3e1f7b4
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
@@ -0,0 +1,136 @@
+/**
+ * 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.physical.impl;
+
+import static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.pop.PopUnitTestBase;
+import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.vector.NullableBigIntVector;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+import java.util.List;
+
+/* This class tests the existing date types. Simply using date types
+ * by casting from VarChar, performing basic functions and converting
+ * back to VarChar.
+ */
+public class TestExtractFunctions extends PopUnitTestBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExtractFunctions.class);
+
+  @Test
+  public void testFromDate() throws Exception {
+    long expectedValues[][] = { {00, 00, 00, 02, 01, 1970}, {00, 00, 00, 28, 12, 2008}, {00, 00, 00, 27, 02, 2000} };
+    testFrom("date", "/test_simple_date.json", "stringdate", expectedValues);
+  }
+
+  @Test
+  @Ignore // failing due to some issue in castTime(varchar)
+  public void testFromTime() throws Exception {
+    long expectedValues[][] = { {33, 20, 10, 00, 00, 0000}, {00, 34, 11, 00, 00, 0000}, {00, 24, 14, 00, 00, 0000} };
+    testFrom("time", "/test_simple_time.json", "stringtime", expectedValues);
+  }
+
+  @Test
+  public void testFromTimeStamp() throws Exception {
+    long expectedValues[][] = { {33, 20, 10, 02, 01, 1970}, {00, 34, 11, 28, 12, 2008}, {00, 24, 14, 27, 02, 2000} };
+    testFrom("timestamp", "/test_simple_date.json", "stringdate", expectedValues);
+  }
+
+  @Test
+  public void testFromInterval() throws Exception {
+    long expectedValues[][] = {
+      { 35, 20, 01, 01, 02, 02},
+      { 00, 00, 00, 00, 02, 02},
+      { 35, 20, 01, 00, 00, 00},
+      { 35, 20, 01, 01, 02, 02},
+      { 35, 00, 00, 00, 00, 00},
+      {-25,-39, 00, 01, 10, 01}
+    };
+    testFrom("interval", "/test_simple_interval.json", "stringinterval", expectedValues);
+  }
+
+  @Test
+  public void testFromIntervalDay() throws Exception {
+    long expectedValues[][] = {
+      { 35, 20, 01, 01, 00, 00},
+      { 00, 00, 00, 00, 00, 00},
+      { 35, 20, 01, 00, 00, 00},
+      { 35, 20, 01, 01, 00, 00},
+      { 35, 00, 00, 00, 00, 00},
+      {-25,-39, 00, 01, 00, 00}
+    };
+    testFrom("intervalday", "/test_simple_interval.json", "stringinterval", expectedValues);
+  }
+
+  @Test
+  public void testFromIntervalYear() throws Exception {
+    long expectedValues[][] = {
+      { 00, 00, 00, 00, 02, 02},
+      { 00, 00, 00, 00, 02, 02},
+      { 00, 00, 00, 00, 00, 00},
+      { 00, 00, 00, 00, 02, 02},
+      { 00, 00, 00, 00, 00, 00},
+      { 00, 00, 00, 00, 10, 01}
+    };
+    testFrom("intervalyear", "/test_simple_interval.json", "stringinterval", expectedValues);
+  }
+
+  private void testFrom(String fromType, String testDataFile, String columnName,
+      long expectedValues[][]) throws Exception {
+    try (RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+         Drillbit bit = new Drillbit(CONFIG, serviceSet);
+         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
+
+      // run query.
+      bit.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+        Files.toString(FileUtils.getResourceAsFile("/functions/extractFrom.json"), Charsets.UTF_8)
+        .replace("#{TEST_TYPE}", fromType)
+        .replace("#{TEST_FILE}", testDataFile)
+        .replace("#{COLUMN_NAME}", columnName));
+
+      RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
+
+      QueryResultBatch batch = results.get(0);
+      assertTrue(batchLoader.load(batch.getHeader().getDef(), batch.getData()));
+
+      for(int i=0; i<expectedValues.length; i++) {
+        for(int j=0; j<expectedValues[i].length; j++) {
+          NullableBigIntVector vv =
+              (NullableBigIntVector) batchLoader.getValueAccessorById(j, NullableBigIntVector.class).getValueVector();
+          System.out.println("["+i+"]["+j+"]: Expected: " + expectedValues[i][j] + ", Actual: " + vv.getAccessor().get(i));
+          assertEquals(expectedValues[i][j], vv.getAccessor().get(i));
+        }
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/test/resources/functions/extractFrom.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/extractFrom.json b/exec/java-exec/src/test/resources/functions/extractFrom.json
new file mode 100644
index 0000000..779581e
--- /dev/null
+++ b/exec/java-exec/src/test/resources/functions/extractFrom.json
@@ -0,0 +1,46 @@
+{
+  "head" : {
+    "version" : 1,
+    "generator" : {
+      "type" : "org.apache.drill.exec.planner.logical.DrillImplementor",
+      "info" : ""
+    },
+    "type" : "APACHE_DRILL_PHYSICAL",
+    "resultMode" : "EXEC"
+  },
+  graph:[
+        {
+            @id:1,
+            pop:"fs-scan",
+            format: {type: "json"},
+            storage:{type: "file", connection: "classpath:///"},
+            files:["#{TEST_FILE}"]
+        },
+        {
+            pop:"project",
+            @id:2,
+            child: 1,
+            exprs: [ {
+              ref: "castExp", expr: "cast(#{COLUMN_NAME} as #{TEST_TYPE})"
+            } ]
+        },
+        {
+            pop:"project",
+            @id:3,
+            child: 2,
+            exprs: [
+              { ref: "extractSecond", expr: "extractSecond(castExp)" },
+              { ref: "extractMinute", expr: "extractMinute(castExp)" },
+              { ref: "extractHour", expr: "extractHour(castExp)" },
+              { ref: "extractDay", expr: "extractDay(castExp)" },
+              { ref: "extractMonth", expr: "extractMonth(castExp)" },
+              { ref: "extractYear", expr: "extractYear(castExp)" }
+            ]
+        },
+        {
+            @id: 4,
+            child: 3,
+            pop: "screen"
+        }
+    ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/dcc102a2/exec/java-exec/src/test/resources/test_simple_time.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/test_simple_time.json b/exec/java-exec/src/test/resources/test_simple_time.json
new file mode 100644
index 0000000..1f8b34f
--- /dev/null
+++ b/exec/java-exec/src/test/resources/test_simple_time.json
@@ -0,0 +1,3 @@
+{ "stringtime" : "10:20:33"}
+{ "stringtime" : "11:34:00.129"}
+{ "stringtime" : "14:24:00"}


[03/51] [abbrv] git commit: Updates for latest Optiq version

Posted by ja...@apache.org.
Updates for latest Optiq version


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

Branch: refs/heads/master
Commit: a4a2c533c571f6c0e1b1de82bc4ad1aa299b1cae
Parents: 62a7866
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Mar 31 10:06:30 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:08 2014 -0700

----------------------------------------------------------------------
 .../exec/store/hbase/HBaseSchemaFactory.java    | 16 +++---
 .../apache/drill/exec/store/AbstractSchema.java | 22 +++------
 .../apache/drill/exec/store/SchemaHolder.java   | 44 -----------------
 .../drill/exec/store/StoragePluginRegistry.java | 52 ++++++--------------
 .../exec/store/dfs/FileSystemSchemaFactory.java | 18 +++----
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |  9 ++--
 .../store/hive/schema/HiveDatabaseSchema.java   |  4 +-
 .../store/hive/schema/HiveSchemaFactory.java    | 18 +++----
 .../store/ischema/InfoSchemaStoragePlugin.java  |  8 ++-
 .../drill/exec/store/ischema/OptiqProvider.java |  2 +-
 pom.xml                                         | 14 ++++++
 .../org/apache/drill/exec/proto/UserProtos.java | 29 +++++++++--
 protocol/src/main/protobuf/User.proto           |  2 -
 13 files changed, 88 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index d97f3d0..1376210 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -21,17 +21,18 @@ import java.io.IOException;
 import java.util.Collections;
 import java.util.Set;
 
-import com.google.common.collect.Sets;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
+
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
-import org.apache.drill.exec.store.SchemaHolder;
 import org.apache.hadoop.hbase.HTableDescriptor;
 import org.apache.hadoop.hbase.client.HBaseAdmin;
 
+import com.google.common.collect.Sets;
+
 public class HBaseSchemaFactory implements SchemaFactory {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseSchemaFactory.class);
 
@@ -45,22 +46,19 @@ public class HBaseSchemaFactory implements SchemaFactory {
 
   @Override
   public Schema add(SchemaPlus parent) {
-    HBaseSchema schema = new HBaseSchema(new SchemaHolder(parent), schemaName);
-    SchemaPlus hPlus = parent.add(schema);
+    HBaseSchema schema = new HBaseSchema(schemaName);
+    SchemaPlus hPlus = parent.add(schemaName, schema);
     schema.setHolder(hPlus);
     return schema;
   }
 
   class HBaseSchema extends AbstractSchema {
 
-    private final SchemaHolder holder = new SchemaHolder();
-
-    public HBaseSchema(SchemaHolder parentSchema, String name) {
-      super(parentSchema, name);
+    public HBaseSchema(String name) {
+      super(name);
     }
 
     public void setHolder(SchemaPlus plusOfThis) {
-      holder.setSchema(plusOfThis);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/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 6aad394..64087f5 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
@@ -32,24 +32,14 @@ import org.apache.drill.exec.planner.logical.DrillTable;
 public abstract class AbstractSchema implements Schema{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(AbstractSchema.class);
 
-  private final SchemaHolder parentSchema;
-
   protected final String name;
   private static final Expression EXPRESSION = new DefaultExpression(Object.class);
 
-  public AbstractSchema(SchemaHolder parentSchema, String name) {
+  public AbstractSchema(String name) {
     super();
-    this.parentSchema = parentSchema;
     this.name = name;
   }
 
-  
-  @Override
-  public SchemaPlus getParentSchema() {
-    return parentSchema.getSchema();
-  }
-
-  @Override
   public String getName() {
     return name;
   }
@@ -75,11 +65,6 @@ public abstract class AbstractSchema implements Schema{
   }
 
   @Override
-  public Expression getExpression() {
-    return EXPRESSION;
-  }
-
-  @Override
   public boolean isMutable() {
     return false;
   }
@@ -93,6 +78,11 @@ public abstract class AbstractSchema implements Schema{
   public Set<String> getTableNames() {
     return Collections.emptySet();
   }
+
+  @Override
+  public Expression getExpression(SchemaPlus parentSchema, String name) {
+    return EXPRESSION;
+  }
   
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaHolder.java
deleted file mode 100644
index faaca2a..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaHolder.java
+++ /dev/null
@@ -1,44 +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;
-
-import net.hydromatic.optiq.SchemaPlus;
-
-/**
- * Helper class to provide parent schema after initialization given Optiq's backwards schema build model.
- */
-public class SchemaHolder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemaHolder.class);
-  
-  private SchemaPlus schema;
-
-  public SchemaHolder(){}
-  
-  public SchemaHolder(SchemaPlus schema){
-    this.schema = schema;
-  }
-  
-  public SchemaPlus getSchema() {
-    return schema;
-  }
-
-  public void setSchema(SchemaPlus schema) {
-    this.schema = schema;
-  }
-   
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/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 b9d8f33..3594028 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
@@ -51,6 +51,7 @@ import org.apache.drill.exec.store.ischema.InfoSchemaStoragePlugin;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
 import com.google.common.io.Resources;
 
 
@@ -63,6 +64,8 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
   private DrillbitContext context;
   private final DrillSchemaFactory schemaFactory = new DrillSchemaFactory();
   
+  private static final Expression EXPRESSION = new DefaultExpression(Object.class);
+  
   public StoragePluginRegistry(DrillbitContext context) {
     try{
     this.context = context;
@@ -201,16 +204,6 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public SchemaPlus getParentSchema() {
-      return inner.getParentSchema();
-    }
-
-    @Override
-    public String getName() {
-      return inner.getName();
-    }
-
-    @Override
     public Table getTable(String name) {
       return inner.getTable(name);
     }
@@ -234,24 +227,12 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     public Set<String> getSubSchemaNames() {
       return inner.getSubSchemaNames();
     }
-
-    @Override
-    public Expression getExpression() {
-      return inner.getExpression();
-    }
-
-    @Override
-    public SchemaPlus addRecursive(Schema schema) {
-      return schema.getParentSchema().add(schema);
-    }
-    
-    
     
   }
 
   private class OrphanPlus implements SchemaPlus{
 
-    private HashMap<String, SchemaPlus> schemas = new HashMap();
+    private HashMap<String, SchemaPlus> schemas = Maps.newHashMap();
     
     @Override
     public SchemaPlus getParentSchema() {
@@ -289,23 +270,11 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public Expression getExpression() {
-      return new DefaultExpression(Object.class);
-    }
-
-    @Override
     public SchemaPlus getSubSchema(String name) {
       return schemas.get(name);
     }
 
     @Override
-    public SchemaPlus add(Schema schema) {
-      OrphanPlusWrap plus = new OrphanPlusWrap(schema);
-      schemas.put(schema.getName(), plus);
-      return plus;
-    }
-
-    @Override
     public void add(String name, Table table) {
       throw new UnsupportedOperationException();
     }
@@ -326,10 +295,17 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public SchemaPlus addRecursive(Schema schema) {
-      return schema.getParentSchema().add(schema);
+    public Expression getExpression(SchemaPlus parentSchema, String name) {
+      return EXPRESSION;
     }
-    
+
+    @Override
+    public SchemaPlus add(String name, Schema schema) {
+      OrphanPlusWrap plus = new OrphanPlusWrap(schema);
+      schemas.put(name, plus);
+      return plus;
+    }
+
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index b150cc1..f569098 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -29,7 +29,6 @@ import net.hydromatic.optiq.SchemaPlus;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
-import org.apache.drill.exec.store.SchemaHolder;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 
 import com.google.common.collect.Maps;
@@ -53,8 +52,9 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
   @Override
   public Schema add(SchemaPlus parent) {
-    FileSystemSchema schema = new FileSystemSchema(parent, schemaName);
-    schema.setHolder(parent.add(schema));
+    FileSystemSchema schema = new FileSystemSchema(schemaName);
+    SchemaPlus plusOfThis = parent.add(schema.getName(), schema);
+    schema.setPlus(plusOfThis);
     return schema;
   }
 
@@ -62,22 +62,20 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
     private final WorkspaceSchema defaultSchema;
     private final Map<String, WorkspaceSchema> schemaMap = Maps.newHashMap();
-    final SchemaHolder selfHolder = new SchemaHolder();
     
-    public FileSystemSchema(SchemaPlus parentSchema, String name) {
-      super(new SchemaHolder(parentSchema), name);
+    public FileSystemSchema(String name) {
+      super(name);
       for(WorkspaceSchemaFactory f :  factories){
-        WorkspaceSchema s = f.create(selfHolder);
+        WorkspaceSchema s = f.createSchema();
         schemaMap.put(s.getName(), s);
       }
       
       defaultSchema = schemaMap.get("default");
     }
 
-    void setHolder(SchemaPlus plusOfThis){
-      selfHolder.setSchema(plusOfThis);
+    void setPlus(SchemaPlus plusOfThis){
       for(WorkspaceSchema s : schemaMap.values()){
-        plusOfThis.add(s);
+        plusOfThis.add(s.getName(), s);
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
index 2ce7648..0ea3231 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/WorkspaceSchemaFactory.java
@@ -26,7 +26,6 @@ import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.planner.sql.ExpandingConcurrentMap;
 import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.SchemaHolder;
 import org.apache.drill.exec.store.dfs.shim.DrillFileSystem;
 import org.apache.hadoop.fs.Path;
 
@@ -62,8 +61,8 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
     this.schemaName = schemaName;
   }
 
-  public WorkspaceSchema create(SchemaHolder holder) {
-    return new WorkspaceSchema(holder, schemaName);
+  public WorkspaceSchema createSchema() {
+    return new WorkspaceSchema(schemaName);
   }
 
   @Override
@@ -106,8 +105,8 @@ public class WorkspaceSchemaFactory implements ExpandingConcurrentMap.MapValueFa
 
   public class WorkspaceSchema extends AbstractSchema {
 
-    public WorkspaceSchema(SchemaHolder parentSchema, String name) {
-      super(parentSchema, name);
+    public WorkspaceSchema(String name) {
+      super(name);
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
index e4ff372..3608ea7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveDatabaseSchema.java
@@ -22,7 +22,6 @@ import java.util.Set;
 
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
-import org.apache.drill.exec.store.SchemaHolder;
 import org.apache.drill.exec.store.hive.schema.HiveSchemaFactory.HiveSchema;
 
 import com.google.common.collect.Sets;
@@ -36,9 +35,8 @@ public class HiveDatabaseSchema extends AbstractSchema{
   public HiveDatabaseSchema( //
       List<String> tableList, //
       HiveSchema hiveSchema, //
-      SchemaHolder parentSchema, //
       String name) {
-    super(parentSchema, name);
+    super(name);
     this.hiveSchema = hiveSchema;
     this.tables = Sets.newHashSet(tableList);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index 4e80afc..29eda92 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -30,11 +30,8 @@ import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
-import org.apache.drill.exec.store.SchemaHolder;
-import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
 import org.apache.drill.exec.store.hive.HiveStoragePlugin;
-import org.apache.drill.exec.store.hive.HiveStoragePluginConfig;
 import org.apache.drill.exec.store.hive.HiveTable;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.HiveMetaStoreClient;
@@ -174,20 +171,18 @@ public class HiveSchemaFactory implements SchemaFactory {
 
   @Override
   public Schema add(SchemaPlus parent) {
-    HiveSchema schema = new HiveSchema(new SchemaHolder(parent), schemaName);
-    SchemaPlus hPlus = parent.add(schema);
+    HiveSchema schema = new HiveSchema(schemaName);
+    SchemaPlus hPlus = parent.add(schemaName, schema);
     schema.setHolder(hPlus);
     return schema;
   }
 
   class HiveSchema extends AbstractSchema {
 
-    private final SchemaHolder holder = new SchemaHolder();
-
     private HiveDatabaseSchema defaultSchema;
     
-    public HiveSchema(SchemaHolder parentSchema, String name) {
-      super(parentSchema, name);
+    public HiveSchema(String name) {
+      super(name);
       getSubSchema("default");
     }
     
@@ -196,7 +191,7 @@ public class HiveSchemaFactory implements SchemaFactory {
       List<String> tables;
       try {
         tables = tableNameLoader.get(name);
-        HiveDatabaseSchema schema = new HiveDatabaseSchema(tables, this, holder, name);
+        HiveDatabaseSchema schema = new HiveDatabaseSchema(tables, this, name);
         if(name.equals("default")){
           this.defaultSchema = schema;
         }
@@ -210,9 +205,8 @@ public class HiveSchemaFactory implements SchemaFactory {
     
 
     void setHolder(SchemaPlus plusOfThis){
-      holder.setSchema(plusOfThis);
       for(String s : getSubSchemaNames()){
-        plusOfThis.add(getSubSchema(s));
+        plusOfThis.add(s, getSubSchema(s));
       }
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
index a7d7c5c..9cc17bd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
@@ -26,12 +26,10 @@ import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.planner.logical.DrillTable;
 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.SchemaHolder;
 
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -67,15 +65,15 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin{
   
   @Override
   public Schema createAndAddSchema(SchemaPlus parent) {
-    Schema s = new ISchema(parent, this);
-    parent.add(s);
+    ISchema s = new ISchema(parent, this);
+    parent.add(s.getName(), s);
     return s;
   }
   
   private class ISchema extends AbstractSchema{
     private Map<String, InfoSchemaDrillTable> tables;
     public ISchema(SchemaPlus parent, InfoSchemaStoragePlugin plugin){
-      super(new SchemaHolder(parent), "INFORMATION_SCHEMA");
+      super("INFORMATION_SCHEMA");
       Map<String, InfoSchemaDrillTable> tbls = Maps.newHashMap();
       for(SelectedTable tbl : SelectedTable.values()){
         tbls.put(tbl.name(), new InfoSchemaDrillTable(plugin, "INFORMATION_SCHEMA", tbl, config));  

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
index 795b92c..bead67b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
@@ -184,7 +184,7 @@ public class OptiqProvider  {
      * Start scanning an Optiq Schema.
      * @param root - where to start
      */
-    protected void scanSchema(Schema root) {
+    protected void scanSchema(SchemaPlus root) {
       scanSchema(root.getName(), root);
     }
     

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 97cf61d..a6200cd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -57,6 +57,20 @@
     </repository>
 
     <repository>
+      <!--  Keep until we move back to released optiq versions. -->
+      <id>mapr-drill-optiq-snapshots</id>
+      <name>MapR Drill Optiq Snapshots</name>
+      <url>http://repository.mapr.com/nexus/content/repositories/drill-optiq/</url>
+      <releases>
+        <enabled>false</enabled>
+      </releases>
+      <snapshots>
+        <enabled>true</enabled>
+      </snapshots>
+    </repository>
+    
+
+    <repository>
       <id>pentaho</id>
       <name>Pentaho</name>
       <url>http://repo.pentaho.org/artifactory/repo</url>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index 8d4fc60..b8efa84 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -74,6 +74,14 @@ public final class UserProtos {
      * <code>QUERY_HANDLE = 7;</code>
      */
     QUERY_HANDLE(7, 7),
+    /**
+     * <code>REQ_META_FUNCTIONS = 8;</code>
+     */
+    REQ_META_FUNCTIONS(8, 8),
+    /**
+     * <code>RESP_FUNCTION_LIST = 9;</code>
+     */
+    RESP_FUNCTION_LIST(9, 9),
     ;
 
     /**
@@ -116,6 +124,14 @@ public final class UserProtos {
      * <code>QUERY_HANDLE = 7;</code>
      */
     public static final int QUERY_HANDLE_VALUE = 7;
+    /**
+     * <code>REQ_META_FUNCTIONS = 8;</code>
+     */
+    public static final int REQ_META_FUNCTIONS_VALUE = 8;
+    /**
+     * <code>RESP_FUNCTION_LIST = 9;</code>
+     */
+    public static final int RESP_FUNCTION_LIST_VALUE = 9;
 
 
     public final int getNumber() { return value; }
@@ -130,6 +146,8 @@ public final class UserProtos {
         case 5: return REQUEST_RESULTS;
         case 6: return QUERY_RESULT;
         case 7: return QUERY_HANDLE;
+        case 8: return REQ_META_FUNCTIONS;
+        case 9: return RESP_FUNCTION_LIST;
         default: return null;
       }
     }
@@ -5211,14 +5229,15 @@ public final class UserProtos {
       "2\033.exec.shared.RecordBatchDef\022\026\n\016schema_" +
       "changed\030\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000" +
       "\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED" +
-      "\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\210\001\n\007Rp" +
+      "\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\270\001\n\007Rp" +
       "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
       "\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017R" +
       "EQUEST_RESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QU" +
-      "ERY_HANDLE\020\007*/\n\tQueryType\022\007\n\003SQL\020\001\022\013\n\007LO" +
-      "GICAL\020\002\022\014\n\010PHYSICAL\020\003*#\n\020QueryResultsMod" +
-      "e\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache.drill.e",
-      "xec.protoB\nUserProtosH\001"
+      "ERY_HANDLE\020\007\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022" +
+      "RESP_FUNCTION_LIST\020\t*/\n\tQueryType\022\007\n\003SQL" +
+      "\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*#\n\020QueryRe",
+      "sultsMode\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache" +
+      ".drill.exec.protoB\nUserProtosH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/a4a2c533/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index fb4e093..3bbe039 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -48,8 +48,6 @@ message RunQuery {
   optional string plan = 3;
 }
 
-message 
-
 enum QueryType {
   SQL = 1;
   LOGICAL = 2;


[06/51] [abbrv] DRILL-450: Add exchange rules, move from BasicOptimizer to Optiq

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
index 585f9a0..0c9bf34 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -1,20 +1,85 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 package org.apache.drill.exec.planner.physical;
 
-import org.apache.drill.common.expression.LogicalExpression;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.common.logical.PlanProperties;
+import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.common.logical.PlanProperties.PlanPropertiesBuilder;
+import org.apache.drill.common.logical.PlanProperties.PlanType;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rex.RexNode;
+
+import com.google.common.collect.Lists;
+
 
 public class PhysicalPlanCreator {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanCreator.class);
+
+  private List<PhysicalOperator> popList;
+  private final QueryContext context;
+  PhysicalPlan plan = null;
   
-  public LogicalExpression getExpression(RexNode e, RelNode input){
-    return null;
+  public PhysicalPlanCreator(QueryContext context) {
+    this.context = context;
+    popList = Lists.newArrayList();
   }
   
-  public DrillParseContext getContext(){
-    return null;
+  public QueryContext getContext() {
+    return context;
   }
   
+  public void addPhysicalOperator(PhysicalOperator op) {
+    popList.add(op);  
+  }
   
+  public PhysicalPlan build(Prel rootPrel, boolean forceRebuild) {
+
+    if (plan != null && !forceRebuild) {
+      return plan;
+    }
+    
+    PlanPropertiesBuilder propsBuilder = PlanProperties.builder();
+    propsBuilder.type(PlanType.APACHE_DRILL_PHYSICAL);
+    propsBuilder.version(1);
+    propsBuilder.resultMode(ResultMode.EXEC);
+    propsBuilder.generator(PhysicalPlanCreator.class.getName(), "");
+
+    
+    try { 
+      // invoke getPhysicalOperator on the root Prel which will recursively invoke it 
+      // on the descendants and we should have a well-formed physical operator tree
+      PhysicalOperator rootPOP = rootPrel.getPhysicalOperator(this);
+      if (rootPOP != null) {
+        assert (popList.size() > 0); //getPhysicalOperator() is supposed to populate this list 
+        plan = new PhysicalPlan(propsBuilder.build(), popList);
+      }
+      
+    } catch (IOException e) {
+      plan = null;
+      throw new UnsupportedOperationException("Physical plan created failed with error : " + e.toString());
+    }
+    
+    return plan;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
index 6640aef..e0dace1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
@@ -1,17 +1,34 @@
+/**
+ * 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.physical;
 
 import java.io.IOException;
 
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.eigenbase.rel.RelNode;
+import org.apache.drill.exec.planner.common.DrillRelNode;
 import org.eigenbase.relopt.Convention;
 
-public interface Prel extends RelNode {
+public interface Prel extends DrillRelNode {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Prel.class);
   
-  final static Convention DRILL_PHYSICAL = new Convention.Impl("DRILL_PHYSICAL", Prel.class);
+  final static Convention DRILL_PHYSICAL = new Convention.Impl("PHYSICAL", Prel.class);
   
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException;
-    
   
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
new file mode 100644
index 0000000..22e8090
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -0,0 +1,72 @@
+/**
+ * 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.physical;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.exec.expr.fn.impl.HashFunctions;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.reltype.RelDataType;
+
+import com.beust.jcommander.internal.Lists;
+import com.google.common.collect.ImmutableList;
+
+public class PrelUtil {
+
+  public static List<Ordering> getOrdering(RelCollation collation, RelDataType rowType) {
+    List<Ordering> orderExpr = Lists.newArrayList();
+    
+    final List<String> childFields = rowType.getFieldNames();
+    
+    for (RelFieldCollation fc: collation.getFieldCollations() ) {      
+      FieldReference fr = new FieldReference(childFields.get(fc.getFieldIndex()), ExpressionPosition.UNKNOWN);
+      orderExpr.add(new Ordering(fc.getDirection(), fr, fc.nullDirection));
+    }
+    
+    return orderExpr;
+  }
+
+  /*
+   * Return a hash expression :  hash(field1) ^ hash(field2) ^ hash(field3) ... ^ hash(field_n)
+   */
+  public static LogicalExpression getHashExpression(List<DistributionField> fields, RelDataType rowType) {
+    assert fields.size() > 0;
+    
+    final List<String> childFields = rowType.getFieldNames();
+    
+    FieldReference fr = new FieldReference(childFields.get(fields.get(0).getFieldId()), ExpressionPosition.UNKNOWN);    
+    FunctionCall func = new FunctionCall("hash",  ImmutableList.of((LogicalExpression)fr), ExpressionPosition.UNKNOWN);
+    
+    for (int i = 1; i<fields.size(); i++) {     
+      fr = new FieldReference(childFields.get(fields.get(i).getFieldId()), ExpressionPosition.UNKNOWN);      
+      FunctionCall func2 = new FunctionCall("hash",  ImmutableList.of((LogicalExpression)fr), ExpressionPosition.UNKNOWN);
+      
+      func = new FunctionCall("^", ImmutableList.of((LogicalExpression)func, (LogicalExpression)func2), ExpressionPosition.UNKNOWN);
+    }
+    
+    return func;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
index b7e18e8..7a9ad43 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -1,3 +1,20 @@
+/**
+ * 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.physical;
 
 import java.io.IOException;
@@ -6,14 +23,18 @@ import java.util.List;
 
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.Project;
-import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.ProjectRelBase;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.rex.RexNode;
 
-public class ProjectPrel extends BaseProjectRel implements Prel{
+public class ProjectPrel extends DrillProjectRelBase implements Prel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPrel.class);
   
   
@@ -22,15 +43,26 @@ public class ProjectPrel extends BaseProjectRel implements Prel{
     super(DRILL_PHYSICAL, cluster, traits, child, exps, rowType);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new ProjectPrel(getCluster(), traitSet, sole(inputs), new ArrayList<RexNode>(exps), rowType);
+  public ProjectRelBase copy(RelTraitSet traitSet, RelNode input, List<RexNode> exps, RelDataType rowType) {
+    return new ProjectPrel(getCluster(), traitSet, input, exps, rowType);
   }
 
+
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    Project p = new Project(this.getProjectExpressions(creator.getContext()), child.getPhysicalOperator(creator));
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, Project only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
+      childPOP = new SelectionVectorRemover(childPOP);
+      creator.addPhysicalOperator(childPOP);
+    }
+    
+    Project p = new Project(this.getProjectExpressions(new DrillParseContext()),  childPOP);
+    creator.addPhysicalOperator(p);
+    
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
index eb1efce..8c6a1e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
@@ -17,7 +17,8 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.eigenbase.rel.ProjectRel;
@@ -26,20 +27,33 @@ import org.eigenbase.relopt.Convention;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.RelOptRuleCall;
 import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.relopt.volcano.RelSubset;
 
 public class ProjectPrule extends RelOptRule {
   public static final RelOptRule INSTANCE = new ProjectPrule();
 
   private ProjectPrule() {
-    super(RelOptHelper.some(BaseProjectRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "ProjectPrule");
+    super(RelOptHelper.some(DrillProjectRel.class, RelOptHelper.any(RelNode.class)), "ProjectPrule");
   }
 
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final BaseProjectRel project = (BaseProjectRel) call.rel(0);
+    final DrillProjectRel project = (DrillProjectRel) call.rel(0);
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = project.getTraitSet().replace(Prel.DRILL_PHYSICAL);
-    final RelNode convertedInput = convert(input, traits);
-    call.transformTo(new ProjectPrel(project.getCluster(), traits, convertedInput, project.getProjects(), project.getRowType()));
+
+    RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL);
+    RelNode convertedInput = convert(input, traits);
+    
+    if (convertedInput instanceof RelSubset) {
+      RelSubset subset = (RelSubset) convertedInput;
+      for (RelNode rel : subset.getRelList()) {
+        if (!rel.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE).equals(DrillDistributionTrait.DEFAULT)) {
+          call.transformTo(new ProjectPrel(project.getCluster(), rel.getTraitSet(), rel, project.getProjects(), project.getRowType()));
+        }
+      }
+      
+    } else{
+      call.transformTo(new ProjectPrel(project.getCluster(), convertedInput.getTraitSet(), convertedInput, project.getProjects(), project.getRowType()));        
+    }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index b4906f1..c411649 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -1,27 +1,62 @@
+/**
+ * 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.physical;
 
 import java.io.IOException;
+import java.util.List;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.exec.physical.base.GroupScan;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
-import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.store.StoragePlugin;
+import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptTable;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class ScanPrel extends BaseScanRel implements Prel{
+public class ScanPrel extends DrillScanRelBase implements Prel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanPrel.class);
 
   public ScanPrel(RelOptCluster cluster, RelTraitSet traits, RelOptTable tbl) {
     super(DRILL_PHYSICAL, cluster, traits, tbl);
   }
 
+  
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return super.copy(traitSet, inputs);
+  }
+
+
+  @Override
+  protected Object clone() throws CloneNotSupportedException {
+    return super.clone();
+  }
+
+
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     StoragePlugin plugin = this.drillTable.getPlugin();
     GroupScan scan = plugin.getPhysicalScan(new JSONOptions(drillTable.getSelection()));
+    creator.addPhysicalOperator(scan);
+    
     return scan;    
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
index 568f8e0..8985a58 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
@@ -17,7 +17,11 @@
  */
 package org.apache.drill.exec.planner.physical;
 
-import org.apache.drill.exec.planner.common.BaseScanRel;
+import java.io.IOException;
+
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.RelOptRuleCall;
@@ -28,14 +32,32 @@ public class ScanPrule extends RelOptRule{
 
   
   public ScanPrule() {
-    super(RelOptHelper.any(BaseScanRel.class), "Prel.ScanRule");
+    super(RelOptHelper.any(DrillScanRelBase.class), "Prel.ScanPrule");
     
   }
   @Override
   public void onMatch(RelOptRuleCall call) {
-    final BaseScanRel scan = (BaseScanRel) call.rel(0);
-    final RelTraitSet traits = scan.getTraitSet().replace(Prel.DRILL_PHYSICAL);
-    call.transformTo(new ScanPrel(scan.getCluster(), traits, scan.getTable()));
+    try{
+      final DrillScanRelBase scan = (DrillScanRelBase) call.rel(0);
+      DrillTable table = scan.getTable().unwrap(DrillTable.class);
+      
+      DrillDistributionTrait partition = table.getGroupScan().getMaxParallelizationWidth() > 1 ? DrillDistributionTrait.RANDOM_DISTRIBUTED : DrillDistributionTrait.SINGLETON;
+
+//      DrillDistributionTrait partition = DrillDistributionTrait.SINGLETON;
+//      
+//      if (table instanceof DynamicDrillTable ) {
+//        if (table.getGroupScan().getMaxParallelizationWidth() > 1 ) 
+//          partition = DrillDistributionTrait.RANDOM_DISTRIBUTED;
+//      }
+      
+      final RelTraitSet traits = scan.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(partition);
+
+      DrillScanRelBase newScan = new ScanPrel(scan.getCluster(), traits, scan.getTable());
+      call.transformTo(newScan);
+      
+    }catch(IOException e){
+      throw new RuntimeException("Failure getting group scan.", e);
+    }
   }
 
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
index 4868c66..6819f31 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
@@ -1,19 +1,41 @@
+/**
+ * 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.physical;
 
+import java.io.IOException;
 import java.util.List;
 
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.planner.common.DrillScreenRelBase;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class ScreenPrel extends SingleRel implements Prel{
+public class ScreenPrel extends DrillScreenRelBase implements Prel {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenPrel.class);
 
   
   public ScreenPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
-    super(cluster, traits, child);
+    super(Prel.DRILL_PHYSICAL, cluster, traits, child);
   }
 
   @Override
@@ -21,5 +43,21 @@ public class ScreenPrel extends SingleRel implements Prel{
     return new ScreenPrel(getCluster(), traitSet, sole(inputs));
   }
   
-  
+  @Override  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, Screen only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
+      childPOP = new SelectionVectorRemover(childPOP);
+      creator.addPhysicalOperator(childPOP);
+    }
+
+    Screen s = new Screen(childPOP, creator.getContext().getCurrentEndpoint());
+    creator.addPhysicalOperator(s);
+    return s; 
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrule.java
new file mode 100644
index 0000000..5b549d5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrule.java
@@ -0,0 +1,49 @@
+/**
+ * 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.physical;
+
+import org.apache.drill.exec.planner.common.DrillScreenRelBase;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class ScreenPrule extends RelOptRule{
+  public static final RelOptRule INSTANCE = new ScreenPrule();
+
+  
+  public ScreenPrule() {
+    super(RelOptHelper.some(DrillScreenRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "Prel.ScreenPrule");    
+  }
+  
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillScreenRelBase screen = (DrillScreenRelBase) call.rel(0);
+    final RelNode input = call.rel(1);
+    
+    final RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    final RelNode convertedInput = convert(input, traits);
+    DrillScreenRelBase newScreen = new ScreenPrel(screen.getCluster(), screen.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), convertedInput);
+    call.transformTo(newScreen);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
new file mode 100644
index 0000000..115d623
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.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.planner.physical;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.Order.Ordering;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.physical.config.SingleMergeExchange;
+import org.apache.drill.exec.physical.config.UnionExchange;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.beust.jcommander.internal.Lists;
+
+public class SingleMergeExchangePrel extends SingleRel implements Prel {
+  
+  private final RelCollation collation ; 
+  
+  public SingleMergeExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelCollation collation) {
+    super(cluster, traitSet, input);
+    this.collation = collation;
+    assert input.getConvention() == Prel.DRILL_PHYSICAL;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+    //return planner.getCostFactory().makeCost(50, 50, 50);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new SingleMergeExchangePrel(getCluster(), traitSet, sole(inputs), collation);
+  }
+  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {    
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
+      childPOP = new SelectionVectorRemover(childPOP);
+      creator.addPhysicalOperator(childPOP);
+    }
+
+    SingleMergeExchange g = new SingleMergeExchange(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()));
+    creator.addPhysicalOperator(g);
+    return g;    
+  }
+    
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
new file mode 100644
index 0000000..f2723cf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortConvertPrule.java
@@ -0,0 +1,57 @@
+/**
+ * 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.physical;
+
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.rel.convert.ConverterRule;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+
+/**
+ * Rule that converts an {@link SortRel} to a physical {@link SortPrel}, implemented by a Drill "order" operation.
+ * 
+ * The {@link SortRel} is added in optiq's AbstractConvert call, when it enforces certain "order" to the input stream. 
+ * Drill uses this rule to convert such sort enforcer into physical {@link SortPrel}. 
+ */
+public class SortConvertPrule extends ConverterRule {
+  public static final RelOptRule INSTANCE = new SortConvertPrule("SortConvertPrule", Convention.NONE);
+  //public static final RelOptRule INSTANCE_SRC_LOGICAL = new SortPrule("SortPrule:Src_Logical", DrillRel.DRILL_LOGICAL);
+
+  private SortConvertPrule(String description, Convention srcConvention) {
+    super(SortRel.class, srcConvention, Prel.DRILL_PHYSICAL, description);
+  }
+
+  @Override
+  public boolean matches(RelOptRuleCall call) {
+    final SortRel sort = call.rel(0);
+    return sort.offset == null && sort.fetch == null;
+  }
+
+  @Override
+  public RelNode convert(RelNode r) {
+    SortRel rel = (SortRel) r;
+    return new SortPrel(rel.getCluster(), 
+                        rel.getChild().getTraitSet().replace(Prel.DRILL_PHYSICAL).plus(rel.getCollation()), 
+                        convert(rel.getChild(), rel.getChild().getTraitSet().replace(Prel.DRILL_PHYSICAL)), 
+                        rel.getCollation());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index 82c7b33..06f6e8b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -1,36 +1,72 @@
+/**
+ * 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.physical;
 
-import java.util.BitSet;
-import java.util.List;
+import java.io.IOException;
 
-import org.apache.drill.exec.planner.logical.DrillAggregateRel;
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.InvalidRelException;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.SingleMergeExchange;
+import org.apache.drill.exec.physical.config.Sort;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
 
-public class SortPrel extends AggregateRelBase{
+public class SortPrel extends SortRel implements Prel {
 
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortPrel.class);
+  /** Creates a DrillSortRel. */
+  public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation) {
+    super(cluster, traits, input, collation);
+  }
 
-  public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
-      List<AggregateCall> aggCalls) throws InvalidRelException {
-    super(cluster, traits, child, groupSet, aggCalls);
-    for (AggregateCall aggCall : aggCalls) {
-      if (aggCall.isDistinct()) {
-        throw new InvalidRelException("DrillAggregateRel does not support DISTINCT aggregates");
-      }
-    }
+  /** Creates a DrillSortRel with offset and fetch. */
+  public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode input, RelCollation collation, RexNode offset, RexNode fetch) {
+    super(cluster, traits, input, collation, offset, fetch);
   }
 
   @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    try {
-      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
-    } catch (InvalidRelException e) {
-      throw new AssertionError(e);
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    if (childPOP.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
+      throw new UnsupportedOperationException();
     }
+    
+    Sort g = new Sort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
+    
+    creator.addPhysicalOperator(g);
+    
+    return g;    
+  }
+
+  public SortPrel copy(
+      RelTraitSet traitSet,
+      RelNode newInput,
+      RelCollation newCollation,
+      RexNode offset,
+      RexNode fetch) {
+    return new SortPrel(getCluster(), traitSet, newInput, newCollation);
   }
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
new file mode 100644
index 0000000..a36450d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrule.java
@@ -0,0 +1,83 @@
+/**
+ * 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.physical;
+
+import java.util.List;
+
+import net.hydromatic.optiq.util.BitSets;
+
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SortRel;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+/**
+ * 
+ * Rule that converts a logical {@link DrillSortRel} to a physical sort.  Convert from Logical Sort into Physical Sort. 
+ * For Logical Sort, it requires one single data stream as the output. 
+ *
+ */
+public class SortPrule extends RelOptRule{
+  public static final RelOptRule INSTANCE = new SortPrule();
+
+  private SortPrule() {
+    super(RelOptHelper.some(DrillSortRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "Prel.SortPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillSortRel sort = (DrillSortRel) call.rel(0);
+    final RelNode input = call.rel(1);
+    
+    // Keep the collation in logical sort. Convert input into a RelNode with 1) this collation, 2) Physical, 3) hash distributed on 
+
+    DrillDistributionTrait hashDistribution = 
+        new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(sort)));
+
+    final RelTraitSet traits = sort.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(hashDistribution);
+    
+    final RelNode convertedInput = convert(input, traits);
+    
+    RelNode exch = new SingleMergeExchangePrel(sort.getCluster(), sort.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), convertedInput, sort.getCollation()); 
+    call.transformTo(exch);  // transform logical "sort" into "SingleMergeExchange".
+    
+  }
+
+  private List<DistributionField> getDistributionField(DrillSortRel rel) {
+    List<DistributionField> distFields = Lists.newArrayList();
+
+    for (RelFieldCollation relField : rel.getCollation().getFieldCollations()) {
+      DistributionField field = new DistributionField(relField.getFieldIndex());
+      distFields.add(field);
+    }    
+    
+    return distFields;
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
deleted file mode 100644
index 820ff48..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
+++ /dev/null
@@ -1,5 +0,0 @@
-package org.apache.drill.exec.planner.physical;
-
-public class StorageRule {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageRule.class);
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index 01b6164..b9f8c41 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -1,9 +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.planner.physical;
 
+import java.io.IOException;
 import java.util.BitSet;
 import java.util.List;
 
+import net.hydromatic.linq4j.Ord;
+import net.hydromatic.optiq.util.BitSets;
+
+import org.apache.drill.common.expression.ExpressionPosition;
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.FunctionCall;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.ValueExpressions;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.SingleMergeExchange;
+import org.apache.drill.exec.physical.config.StreamingAggregate;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.AggregateCall;
 import org.eigenbase.rel.AggregateRelBase;
 import org.eigenbase.rel.InvalidRelException;
@@ -11,7 +44,9 @@ import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class StreamAggPrel extends AggregateRelBase{
+import com.beust.jcommander.internal.Lists;
+
+public class StreamAggPrel extends AggregateRelBase implements Prel{
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamAggPrel.class);
 
@@ -25,12 +60,52 @@ public class StreamAggPrel extends AggregateRelBase{
     }
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  public AggregateRelBase copy(RelTraitSet traitSet, RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
     try {
-      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+      return new StreamAggPrel(getCluster(), traitSet, input, getGroupSet(), aggCalls);
     } catch (InvalidRelException e) {
       throw new AssertionError(e);
     }
   }
+   
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    // Prel child = (Prel) this.getChild();
+    
+    final List<String> childFields = getChild().getRowType().getFieldNames();
+    final List<String> fields = getRowType().getFieldNames();
+    List<NamedExpression> keys = Lists.newArrayList();
+    List<NamedExpression> exprs = Lists.newArrayList();
+    
+    for (int group : BitSets.toIter(groupSet)) {
+      FieldReference fr = new FieldReference(childFields.get(group), ExpressionPosition.UNKNOWN);
+      keys.add(new NamedExpression(fr, fr));
+    }
+    
+    for (Ord<AggregateCall> aggCall : Ord.zip(aggCalls)) {
+      FieldReference ref = new FieldReference(fields.get(groupSet.cardinality() + aggCall.i));
+      LogicalExpression expr = toDrill(aggCall.e, childFields, new DrillParseContext());
+      exprs.add(new NamedExpression(expr, ref));
+    }
+
+    Prel child = (Prel) this.getChild();
+    StreamingAggregate g = new StreamingAggregate(child.getPhysicalOperator(creator), keys.toArray(new NamedExpression[keys.size()]), exprs.toArray(new NamedExpression[exprs.size()]), 1.0f);
+    creator.addPhysicalOperator(g);
+    
+    return g;    
+
+  }
+  
+  private LogicalExpression toDrill(AggregateCall call, List<String> fn, DrillParseContext pContext) {
+    List<LogicalExpression> args = Lists.newArrayList();
+    for(Integer i : call.getArgList()){
+      args.add(new FieldReference(fn.get(i)));
+    }
+    
+    // for count(1).
+    if(args.isEmpty()) args.add(new ValueExpressions.LongExpression(1l));
+    LogicalExpression expr = new FunctionCall(call.getAggregation().getName().toLowerCase(), args, ExpressionPosition.UNKNOWN );
+    return expr;
+  }
+ 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
new file mode 100644
index 0000000..1058401
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrule.java
@@ -0,0 +1,94 @@
+/**
+ * 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.physical;
+
+import java.util.List;
+import java.util.logging.Logger;
+
+import net.hydromatic.optiq.util.BitSets;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelCollationImpl;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.trace.EigenbaseTrace;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class StreamAggPrule extends RelOptRule {
+  public static final RelOptRule INSTANCE = new StreamAggPrule();
+  protected static final Logger tracer = EigenbaseTrace.getPlannerTracer();
+
+  private StreamAggPrule() {
+    super(RelOptHelper.some(DrillAggregateRel.class, RelOptHelper.any(DrillRel.class)), "Prel.StreamAggPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillAggregateRel aggregate = (DrillAggregateRel) call.rel(0);
+    final RelNode input = call.rel(1);
+    RelCollation collation = getCollation(aggregate);
+
+    DrillDistributionTrait hashDistribution = 
+        new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(aggregate)));
+    
+    final RelTraitSet traits = call.getPlanner().emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(collation).plus(hashDistribution);
+    
+    final RelNode convertedInput = convert(input, traits);
+    
+    try {          
+      StreamAggPrel newAgg = new StreamAggPrel(aggregate.getCluster(), traits, convertedInput, aggregate.getGroupSet(),
+          aggregate.getAggCallList());
+      
+      call.transformTo(newAgg);
+    } catch (InvalidRelException e) {
+      tracer.warning(e.toString());
+    }
+  }
+  
+  
+  private RelCollation getCollation(DrillAggregateRel rel){
+    
+    List<RelFieldCollation> fields = Lists.newArrayList();
+    for (int group : BitSets.toIter(rel.getGroupSet())) {
+      fields.add(new RelFieldCollation(group));
+    }
+    return RelCollationImpl.of(fields);
+  }
+
+  private List<DistributionField> getDistributionField(DrillAggregateRel rel) {
+    List<DistributionField> groupByFields = Lists.newArrayList();
+
+    for (int group : BitSets.toIter(rel.getGroupSet())) {
+      DistributionField field = new DistributionField(group);
+      groupByFields.add(field);
+    }    
+    
+    return groupByFields;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
deleted file mode 100644
index 027196e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
+++ /dev/null
@@ -1,36 +0,0 @@
-package org.apache.drill.exec.planner.physical;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.drill.exec.planner.logical.DrillAggregateRel;
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-
-public class TopNSortPrel extends AggregateRelBase{
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopNSortPrel.class);
-
-  public TopNSortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
-      List<AggregateCall> aggCalls) throws InvalidRelException {
-    super(cluster, traits, child, groupSet, aggCalls);
-    for (AggregateCall aggCall : aggCalls) {
-      if (aggCall.isDistinct()) {
-        throw new InvalidRelException("DrillAggregateRel does not support DISTINCT aggregates");
-      }
-    }
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    try {
-      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
-    } catch (InvalidRelException e) {
-      throw new AssertionError(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
new file mode 100644
index 0000000..38812c7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.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.planner.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.physical.config.UnionExchange;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class UnionExchangePrel extends SingleRel implements Prel {
+
+  public UnionExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
+    super(cluster, traitSet, input);
+    assert input.getConvention() == Prel.DRILL_PHYSICAL;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+    //return planner.getCostFactory().makeCost(50, 50, 50);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new UnionExchangePrel(getCluster(), traitSet, sole(inputs));
+  }
+  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+     
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
+      childPOP = new SelectionVectorRemover(childPOP);
+      creator.addPhysicalOperator(childPOP);
+    }
+   
+    UnionExchange g = new UnionExchange(childPOP);
+    creator.addPhysicalOperator(g);
+    return g;    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
new file mode 100644
index 0000000..f20ca99
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.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.planner.sql;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.eigenbase.sql.SqlFunctionCategory;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.sql.SqlOperatorTable;
+import org.eigenbase.sql.SqlSyntax;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
+
+import com.google.common.collect.ArrayListMultimap;
+import com.google.common.collect.Lists;
+
+public class DrillOperatorTable extends SqlStdOperatorTable{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
+
+  private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
+  private List<SqlOperator> operators;
+  private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
+  
+  public DrillOperatorTable(FunctionImplementationRegistry registry){
+    operators = Lists.newArrayList();
+    operators.addAll(inner.getOperatorList());
+    
+    for(Map.Entry<String, DrillFuncHolder> function : registry.getDrillRegistry().getMethods().entries()){
+      SqlOperator op = new DrillSqlOperator(function.getKey(), function.getValue().getParmSize());
+      operators.add(op);
+      opMap.put(function.getKey(), op);
+    }
+    
+    // TODO: add hive functions.
+  }
+  
+  @Override
+  public List<SqlOperator> lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax) {
+    if(syntax == SqlSyntax.FUNCTION){
+      List<SqlOperator> drillOps = opMap.get(opName.getSimple());
+      if(drillOps == null || drillOps.isEmpty()) return inner.lookupOperatorOverloads(opName, category, syntax);
+      
+      List<SqlOperator> optiqOps = inner.lookupOperatorOverloads(opName, category, syntax);
+      if(optiqOps.isEmpty()) return drillOps;
+      
+      // combine the two.
+      List<SqlOperator> both = Lists.newArrayList();
+      both.addAll(optiqOps);
+      both.addAll(drillOps);
+      
+      return both;
+
+    }else{
+      return inner.lookupOperatorOverloads(opName, category, syntax);
+    }
+  }
+
+  @Override
+  public List<SqlOperator> getOperatorList() {
+    return operators;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
new file mode 100644
index 0000000..f09e1a1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
@@ -0,0 +1,155 @@
+/**
+ * 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;
+
+import java.util.Collection;
+import java.util.List;
+
+import org.apache.drill.exec.expr.fn.DrillFuncHolder;
+import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
+import org.apache.drill.exec.planner.types.RelDataTypeHolder;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelProtoDataType;
+import org.eigenbase.sql.SqlCall;
+import org.eigenbase.sql.SqlCallBinding;
+import org.eigenbase.sql.SqlFunction;
+import org.eigenbase.sql.SqlFunctionCategory;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlKind;
+import org.eigenbase.sql.SqlOperandCountRange;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.sql.SqlSyntax;
+import org.eigenbase.sql.type.ExplicitReturnTypeInference;
+import org.eigenbase.sql.type.SqlOperandCountRanges;
+import org.eigenbase.sql.type.SqlOperandTypeChecker;
+import org.eigenbase.sql.type.SqlTypeName;
+import org.eigenbase.sql.validate.SqlValidator;
+import org.eigenbase.sql.validate.SqlValidatorScope;
+
+import com.google.hive12.common.collect.Lists;
+
+public class DrillSqlOperator extends SqlFunction {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
+
+  DrillSqlOperator(String name, int argCount) {
+    super(name, SqlKind.OTHER_FUNCTION, DYNAMIC_RETURN, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
+  }
+
+  private static final DynamicReturnType DYNAMIC_RETURN = new DynamicReturnType();
+
+
+  @Override
+  public SqlIdentifier getSqlIdentifier() {
+    return super.getSqlIdentifier();
+  }
+
+  @Override
+  public SqlIdentifier getNameAsId() {
+    return super.getNameAsId();
+  }
+
+  @Override
+  public List<RelDataType> getParamTypes() {
+    return super.getParamTypes();
+  }
+
+  @Override
+  public SqlFunctionCategory getFunctionType() {
+    return super.getFunctionType();
+  }
+
+  @Override
+  public boolean isQuantifierAllowed() {
+    return super.isQuantifierAllowed();
+  }
+
+  @Override
+  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
+    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), validator.getTypeFactory());
+//    return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
+    //return super.deriveType(validator, scope, call);
+  }
+
+  
+  private static class DynamicReturnType extends ExplicitReturnTypeInference {
+    public DynamicReturnType() {
+      super(new DynamicType());
+    }
+  }
+  private static class DynamicType implements RelProtoDataType {
+
+    @Override
+    public RelDataType apply(RelDataTypeFactory factory) {
+      return factory.createSqlType(SqlTypeName.ANY);
+      //return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
+    }
+
+  }
+
+  private static class FixedRange implements SqlOperandCountRange{
+
+    private final int size;
+    
+    public FixedRange(int size) {
+      super();
+      this.size = size;
+    }
+
+    @Override
+    public boolean isValidCount(int count) {
+      return count == size;
+    }
+
+    @Override
+    public int getMin() {
+      return size;
+    }
+
+    @Override
+    public int getMax() {
+      return size;
+    }
+    
+  }
+  
+  private static class Checker implements SqlOperandTypeChecker {
+    private SqlOperandCountRange range;
+
+    public Checker(int size) {
+      range = new FixedRange(size);
+    }
+
+    @Override
+    public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+      return true;
+    }
+
+    @Override
+    public SqlOperandCountRange getOperandCountRange() {
+      return range;
+    }
+
+    @Override
+    public String getAllowedSignatures(SqlOperator op, String opName) {
+      return opName + "(Drill - Opaque)";
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index ef1529c..676a1e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -17,34 +17,53 @@
  */
 package org.apache.drill.exec.planner.sql;
 
-import net.hydromatic.optiq.jdbc.ConnectionConfig;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import net.hydromatic.optiq.config.Lex;
 import net.hydromatic.optiq.tools.Frameworks;
 import net.hydromatic.optiq.tools.Planner;
 import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.RuleSet;
 import net.hydromatic.optiq.tools.ValidationException;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.logical.LogicalPlan;
 import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.exec.client.QuerySubmitter;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.planner.common.BaseScreenRel;
-import org.apache.drill.exec.planner.common.DrillStoreRel;
+import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
+import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.drill.exec.store.StoragePluginRegistry.DrillSchemaFactory;
+import org.eigenbase.rel.RelCollationTraitDef;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.ConventionTraitDef;
+import org.eigenbase.relopt.RelOptUtil;
+import org.eigenbase.relopt.RelTraitDef;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.sql.SqlExplain;
+import org.eigenbase.sql.SqlExplainLevel;
 import org.eigenbase.sql.SqlKind;
 import org.eigenbase.sql.SqlLiteral;
 import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.sql.parser.SqlParseException;
 
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.common.collect.ImmutableList;
+
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
 
@@ -53,8 +72,14 @@ public class DrillSqlWorker {
   private final static int LOGICAL_RULES = 0;
   private final static int PHYSICAL_MEM_RULES = 1;
   
-  public DrillSqlWorker(DrillSchemaFactory schemaFactory) throws Exception {
-    this.planner = Frameworks.getPlanner(ConnectionConfig.Lex.MYSQL, schemaFactory, SqlStdOperatorTable.instance(), new RuleSet[]{DrillRuleSets.DRILL_BASIC_RULES});
+  public DrillSqlWorker(DrillSchemaFactory schemaFactory, FunctionImplementationRegistry registry) throws Exception {
+    final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+    traitDefs.add(ConventionTraitDef.INSTANCE);
+    traitDefs.add(DrillDistributionTraitDef.INSTANCE);    
+    traitDefs.add(RelCollationTraitDef.INSTANCE);
+    
+    this.planner = Frameworks.getPlanner(Lex.MYSQL, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
+//    this.planner = Frameworks.getPlanner(Lex.MYSQL, SqlParserImpl.FACTORY, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
   }
   
   private class RelResult{
@@ -66,21 +91,23 @@ public class DrillSqlWorker {
       this.node = node;
     }
   }
-  
+
+  /*
+   * Return the logical DrillRel tree 
+   */
   private RelResult getRel(String sql) throws SqlParseException, ValidationException, RelConversionException{
     SqlNode sqlNode = planner.parse(sql);
-
+    
     ResultMode resultMode = ResultMode.EXEC;
+    
     if(sqlNode.getKind() == SqlKind.EXPLAIN){
       SqlExplain explain = (SqlExplain) sqlNode;
-      sqlNode = explain.operands[0];
-      SqlLiteral op = (SqlLiteral) explain.operands[2];
-      SqlExplain.Depth depth = (SqlExplain.Depth) op.getValue();
+      SqlExplain.Depth depth = (SqlExplain.Depth) explain.getDepth();
       switch(depth){
-      case Logical:
+      case LOGICAL:
         resultMode = ResultMode.LOGICAL;
         break;
-      case Physical:
+      case PHYSICAL:
         resultMode = ResultMode.PHYSICAL;
         break;
       default:
@@ -89,13 +116,21 @@ public class DrillSqlWorker {
     
     SqlNode validatedNode = planner.validate(sqlNode);
     RelNode relNode = planner.convert(validatedNode);
-    return new RelResult(resultMode, relNode);
+    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
+    if(convertedRelNode instanceof DrillStoreRel){
+      throw new UnsupportedOperationException();
+    }else{
+      convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
+    }
+
+    return new RelResult(resultMode, convertedRelNode);
   }
   
   
   
   public LogicalPlan getLogicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
     RelResult result = getRel(sql);
+
     RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
@@ -109,15 +144,43 @@ public class DrillSqlWorker {
     return implementor.getPlan();
     
   }
-
   
-  public PhysicalPlan getPhysicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
+  public PhysicalPlan getPhysicalPlan(String sql, QueryContext qcontext) throws SqlParseException, ValidationException, RelConversionException, IOException {
     RelResult result = getRel(sql);
-    RelTraitSet traits = planner.getEmptyTraitSet().plus(Prel.DRILL_PHYSICAL);
-    RelNode transformed = planner.transform(PHYSICAL_MEM_RULES, traits, result.node);
+
+    RelTraitSet traits = result.node.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);    
+    Prel phyRelNode = (Prel) planner.transform(PHYSICAL_MEM_RULES, traits, result.node);
+    
+    //Debug.
+    System.err.println("SQL : " + sql);
+    logger.debug("SQL : " + sql);
+    String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
+    System.out.println(msg);
+    logger.debug(msg);
+        
+    PhysicalPlanCreator pplanCreator = new PhysicalPlanCreator(qcontext);
+    PhysicalPlan plan = pplanCreator.build(phyRelNode, true /* rebuild */);
+        
     planner.close();
     planner.reset();
-    return null;
+    return plan;
+
   }
-  
+ 
+  public void runPhysicalPlan(PhysicalPlan phyPlan, DrillConfig config) {
+    QuerySubmitter qs = new QuerySubmitter();
+    
+    ObjectMapper mapper = config.getMapper();
+    
+    try {
+      String phyPlanStr = mapper.writeValueAsString(phyPlan);
+      
+      System.out.println(phyPlanStr);
+      
+      qs.submitQuery(null, phyPlanStr, "physical", null, true, 1, "csv");
+    } catch (Exception e) {
+      System.err.println("Query fails " + e.toString());
+    }
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
index 470a33d..7456214 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/torel/ConversionContext.java
@@ -34,15 +34,12 @@ import org.apache.drill.common.logical.data.Project;
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.common.logical.data.Union;
 import org.apache.drill.common.logical.data.visitors.AbstractLogicalVisitor;
-import org.apache.drill.exec.planner.common.BaseFilterRel;
-import org.apache.drill.exec.planner.common.DrillLimitRel;
-import org.apache.drill.exec.planner.common.BaseProjectRel;
-import org.apache.drill.exec.planner.common.BaseScanRel;
-import org.apache.drill.exec.planner.common.DrillUnionRel;
 import org.apache.drill.exec.planner.logical.DrillAggregateRel;
 import org.apache.drill.exec.planner.logical.DrillJoinRel;
+import org.apache.drill.exec.planner.logical.DrillLimitRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillSortRel;
+import org.apache.drill.exec.planner.logical.DrillUnionRel;
 import org.apache.drill.exec.planner.logical.ScanFieldDeterminer;
 import org.apache.drill.exec.planner.logical.ScanFieldDeterminer.FieldList;
 import org.eigenbase.rel.InvalidRelException;
@@ -121,17 +118,20 @@ public class ConversionContext implements ToRelContext {
 
     @Override
     public RelNode visitScan(Scan scan, ConversionContext context){
-      return BaseScanRel.convert(scan, context);
+      //return BaseScanRel.convert(scan, context);
+      return null;
     }
 
     @Override
     public RelNode visitFilter(Filter filter, ConversionContext context) throws InvalidRelException{
-      return BaseFilterRel.convert(filter, context);
+      //return BaseFilterRel.convert(filter, context);
+      return null;
     }
 
     @Override
     public RelNode visitProject(Project project, ConversionContext context) throws InvalidRelException{
-      return BaseProjectRel.convert(project, context);
+      //return BaseProjectRel.convert(project, context);
+      return null;
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
new file mode 100644
index 0000000..8b031ec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
@@ -0,0 +1,86 @@
+/**
+ * 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.types;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.reltype.RelDataTypeImpl;
+import org.eigenbase.reltype.RelDataTypePrecedenceList;
+import org.eigenbase.sql.type.SqlTypeExplicitPrecedenceList;
+import org.eigenbase.sql.type.SqlTypeName;
+
+/* We use an instance of this class as the row type for
+ * Drill table. Since we don't know the schema before hand
+ * whenever optiq requires us to validate that a field exists
+ * we always return true and indicate that the type of that
+ * field is 'ANY'
+ */
+public class RelDataTypeDrillImpl extends RelDataTypeImpl {
+
+    private final RelDataTypeFactory typeFactory;
+    private final RelDataTypeHolder holder;
+    
+    public RelDataTypeDrillImpl(RelDataTypeHolder holder, RelDataTypeFactory typeFactory) {
+        this.typeFactory = typeFactory;
+        this.holder = holder;
+        computeDigest();
+    }
+    
+    @Override
+    public List<RelDataTypeField> getFieldList() {
+      return holder.getFieldList(typeFactory);
+    }
+    
+    @Override
+    public int getFieldCount() {
+      return holder.getFieldCount();
+    }
+
+    @Override
+    public RelDataTypeField getField(String fieldName, boolean caseSensitive) {
+      return holder.getField(typeFactory, fieldName);
+    }
+
+    @Override
+    public List<String> getFieldNames() {
+      return holder.getFieldNames();
+    }
+
+    @Override
+    public SqlTypeName getSqlTypeName() {
+        return SqlTypeName.ANY;
+    }
+
+    @Override
+    public RelDataTypePrecedenceList getPrecedenceList() {
+      return new SqlTypeExplicitPrecedenceList((List<SqlTypeName>) (List) Collections.emptyList());
+    }
+
+    @Override
+    protected void generateTypeString(StringBuilder sb, boolean withDetail) {
+       sb.append("DrillRecordRow");
+    }
+
+    @Override
+    public boolean isStruct() {
+        return true;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
new file mode 100644
index 0000000..c60f83a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
@@ -0,0 +1,86 @@
+/**
+ * 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.types;
+
+import java.util.List;
+
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelDataTypeField;
+import org.eigenbase.reltype.RelDataTypeFieldImpl;
+import org.eigenbase.sql.type.SqlTypeName;
+
+import com.google.common.collect.Lists;
+
+public class RelDataTypeHolder {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RelDataTypeHolder.class);
+
+  List<String> fieldNames = Lists.newArrayList();
+
+  public List<RelDataTypeField> getFieldList(RelDataTypeFactory typeFactory) {
+
+    addStarIfEmpty();
+
+    List<RelDataTypeField> fields = Lists.newArrayList();
+
+    int i = 0;
+    for (String fieldName : fieldNames) {
+
+//      RelDataTypeField field = new RelDataTypeFieldImpl(fieldName, i, new RelDataTypeDrillImpl(new RelDataTypeHolder(), typeFactory));
+      RelDataTypeField field = new RelDataTypeFieldImpl(fieldName, i, typeFactory.createSqlType(SqlTypeName.ANY));
+      fields.add(field);
+      i++;
+    }
+
+    return fields;
+  }
+
+  public int getFieldCount() {
+    addStarIfEmpty();
+    return fieldNames.size();
+  }
+
+  private void addStarIfEmpty(){
+    //if (fieldNames.isEmpty()) fieldNames.add("*");
+  }
+  
+  public RelDataTypeField getField(RelDataTypeFactory typeFactory, String fieldName) {
+
+    /* First check if this field name exists in our field list */
+    int i = 0;
+    for (String name : fieldNames) {
+      if (name.equalsIgnoreCase(fieldName)) {
+        return new RelDataTypeFieldImpl(name, i, typeFactory.createSqlType(SqlTypeName.ANY));
+      }
+      i++;
+    }
+
+    /* This field does not exist in our field list add it */
+    RelDataTypeField newField = new RelDataTypeFieldImpl(fieldName, fieldNames.size(),
+        typeFactory.createSqlType(SqlTypeName.ANY));
+
+    /* Add the name to our list of field names */
+    fieldNames.add(fieldName);
+
+    return newField;
+  }
+
+  public List<String> getFieldNames() {
+    addStarIfEmpty();
+    return fieldNames;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
index 0db3221..159a6f4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
@@ -40,12 +40,12 @@ public abstract class Field {
   }
 
   public MaterializedField getAsMaterializedField(FieldReference ref) {
-    if(ref != null){
-      return MaterializedField.create(new SchemaPath(ref.getPath() + "." + getFieldName(), ExpressionPosition.UNKNOWN), fieldType);  
-    }else{
-      return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);
-    }
-    
+    assert (ref!=null);
+    return MaterializedField.create(new SchemaPath(ref.getPath() + "." + getFieldName(), ExpressionPosition.UNKNOWN), fieldType);  
+  }
+
+  public MaterializedField getAsMaterializedField() {
+      return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);    
   }
 
   public abstract String getFieldName();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/shim/fallback/FallbackFileSystem.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/shim/fallback/FallbackFileSystem.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/shim/fallback/FallbackFileSystem.java
index 5743ca1..340919d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/shim/fallback/FallbackFileSystem.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/shim/fallback/FallbackFileSystem.java
@@ -64,7 +64,7 @@ public class FallbackFileSystem extends DrillFileSystem {
   
   private void addRecursiveStatus(FileStatus parent, List<FileStatus> listToFill) throws IOException {
     if (parent.isDir()) {
-      Path pattern = new Path(parent.getPath(), "/*");
+      Path pattern = new Path(parent.getPath(), "*");
       FileStatus[] sub = fs.globStatus(pattern);
       for(FileStatus s : sub){
         listToFill.add(s);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/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 ac03444..8d502a1 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
@@ -147,6 +147,7 @@ public class JSONRecordReader implements RecordReader {
       // Garbage collect fields never referenced in this batch
       for (Field field : Iterables.concat(currentSchema.removeUnreadFields(), removedFields)) {
         diffSchema.addRemovedField(field);
+        outputMutator.removeField(field.getAsMaterializedField());
       }
 
       if (diffSchema.isChanged()) {


[10/51] [abbrv] git commit: set def recordCount for explain

Posted by ja...@apache.org.
set def recordCount for explain


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

Branch: refs/heads/master
Commit: 6976f92adeaf5e82729a0ff5f8581a0c405a6fec
Parents: d37d924
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Mar 20 01:01:19 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../src/main/java/org/apache/drill/exec/work/foreman/Foreman.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6976f92a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 5d53b7d..858508b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -228,7 +228,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     QueryResult header = QueryResult.newBuilder() //
         .setQueryId(context.getQueryId()) //
         .setRowCount(1) //
-        .setDef(RecordBatchDef.newBuilder().addField(vector.getMetadata()).build()) //
+        .setDef(RecordBatchDef.newBuilder().addField(vector.getMetadata()).setRecordCount(1).build()) //
         .setIsLastChunk(false) //
         .build();
     QueryWritableBatch b1 = new QueryWritableBatch(header, vector.getBuffers());


[15/51] [abbrv] git commit: Add the date expressions in the tpch queries

Posted by ja...@apache.org.
Add the date expressions in the tpch queries


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

Branch: refs/heads/master
Commit: 02b206819f0c3c8b390ee81858315ea9d44fb9ea
Parents: 029c5b8
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Mar 31 16:38:47 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/src/test/resources/queries/tpch/03.sql | 4 ++--
 exec/java-exec/src/test/resources/queries/tpch/05.sql | 4 ++--
 exec/java-exec/src/test/resources/queries/tpch/07.sql | 2 +-
 exec/java-exec/src/test/resources/queries/tpch/12.sql | 4 ++--
 4 files changed, 7 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/02b20681/exec/java-exec/src/test/resources/queries/tpch/03.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/03.sql b/exec/java-exec/src/test/resources/queries/tpch/03.sql
index f856e64..91fa846 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/03.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/03.sql
@@ -14,8 +14,8 @@ where
   c.c_mktsegment = 'HOUSEHOLD'
   and c.c_custkey = o.o_custkey
   and l.l_orderkey = o.o_orderkey
---  and o.o_orderdate < date '1995-03-25'
---  and l.l_shipdate > date '1995-03-25'
+  and o.o_orderdate < date '1995-03-25'
+  and l.l_shipdate > date '1995-03-25'
 
 group by
   l.l_orderkey,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/02b20681/exec/java-exec/src/test/resources/queries/tpch/05.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/05.sql b/exec/java-exec/src/test/resources/queries/tpch/05.sql
index ae82d39..11e413f 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/05.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/05.sql
@@ -19,8 +19,8 @@ where
   and s.s_nationkey = n.n_nationkey
   and n.n_regionkey = r.r_regionkey
   and r.r_name = 'EUROPE'
---  and o.o_orderdate >= date '1997-01-01'
---  and o.o_orderdate < date '1997-01-01' + interval '1' year
+  and o.o_orderdate >= date '1997-01-01'
+  and o.o_orderdate < date '1997-01-01' + interval '1' year
 group by
   n.n_name
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/02b20681/exec/java-exec/src/test/resources/queries/tpch/07.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/07.sql b/exec/java-exec/src/test/resources/queries/tpch/07.sql
index 94bd51c..85eaece 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/07.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/07.sql
@@ -28,7 +28,7 @@ from
         (n1.n_name = 'EGYPT' and n2.n_name = 'UNITED STATES')
         or (n1.n_name = 'UNITED STATES' and n2.n_name = 'EGYPT')
       )
---      and l.l_shipdate between date '1995-01-01' and date '1996-12-31'
+      and l.l_shipdate between date '1995-01-01' and date '1996-12-31'
   ) as shipping
 group by
   supp_nation,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/02b20681/exec/java-exec/src/test/resources/queries/tpch/12.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/12.sql b/exec/java-exec/src/test/resources/queries/tpch/12.sql
index 41bc5dd..65d2c6d 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/12.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/12.sql
@@ -21,8 +21,8 @@ where
   and l.l_shipmode in ('TRUCK', 'REG AIR')
   and l.l_commitdate < l.l_receiptdate
   and l.l_shipdate < l.l_commitdate
---  and l.l_receiptdate >= date '1994-01-01'
---  and l.l_receiptdate < date '1994-01-01' + interval '1' year
+  and l.l_receiptdate >= date '1994-01-01'
+  and l.l_receiptdate < date '1994-01-01' + interval '1' year
 group by
   l.l_shipmode
 order by


[50/51] [abbrv] git commit: Fix EXPLAIN plan which was broken in recent change, add test case for EXPLAIN. Move debug msg in DrillSqlWorker to logger.

Posted by ja...@apache.org.
Fix EXPLAIN plan which was broken in recent change, add test case for EXPLAIN.  Move debug msg in DrillSqlWorker to logger.


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

Branch: refs/heads/master
Commit: 4f98a4f41802771a487671382c3b189cd9660b1b
Parents: 56411a5
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Wed Apr 9 16:18:50 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/planner/sql/DrillSqlWorker.java  | 115 +++++++++++++------
 .../apache/drill/exec/work/foreman/Foreman.java |  18 ++-
 .../org/apache/drill/TestExampleQueries.java    |  10 ++
 3 files changed, 108 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4f98a4f4/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index e796f12..d5ad1fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -60,6 +60,7 @@ import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.parser.SqlParseException;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
+import com.google.hive12.common.base.Preconditions;
 
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
@@ -81,7 +82,7 @@ public class DrillSqlWorker {
 //    this.planner = Frameworks.getPlanner(Lex.MYSQL, SqlParserImpl.FACTORY, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
   }
   
-  private class RelResult{
+  public class RelResult{
     final ResultMode mode;
     final RelNode node;
     public RelResult(ResultMode mode, RelNode node) {
@@ -89,18 +90,28 @@ public class DrillSqlWorker {
       this.mode = mode;
       this.node = node;
     }
+    
+    public ResultMode getMode() {
+      return this.mode;
+    }
   }
 
   /*
-   * Return the logical DrillRel tree 
+   * Given a SQL string, return the logical DrillRel tree, plus mode (execute, or EXPLAIN mode).  
    */
-  private RelResult getRel(String sql) throws SqlParseException, ValidationException, RelConversionException{
-    SqlNode sqlNode = planner.parse(sql);
-    
+  public RelResult getLogicalRel(String sql) throws SqlParseException, ValidationException, RelConversionException{
+    if(logger.isDebugEnabled()) {
+      logger.debug("SQL : " + sql);
+    }
+
+    // Call optiq to parse the SQL string. 
+    SqlNode sqlNode = planner.parse(sql);  
     ResultMode resultMode = ResultMode.EXEC;
     
+    //Process EXPLAIN
     if(sqlNode.getKind() == SqlKind.EXPLAIN){
       SqlExplain explain = (SqlExplain) sqlNode;
+      sqlNode = explain.operand(0);
       SqlExplain.Depth depth = (SqlExplain.Depth) explain.getDepth();
       switch(depth){
       case LOGICAL:
@@ -113,11 +124,16 @@ public class DrillSqlWorker {
       }
     }
     
+    // Call optiq to validate SqlNode tree and convert it to RelNode tree. 
     SqlNode validatedNode = planner.validate(sqlNode);
     RelNode relNode = planner.convert(validatedNode);
     
-    System.out.println(RelOptUtil.toString(relNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    //Debug
+    if(logger.isDebugEnabled()) {
+      logger.debug("RelNode tree : " + RelOptUtil.toString(relNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    }
     
+    // Call optiq to transform RelNode into Drill Logical RelNode tree. 
     RelNode convertedRelNode = planner.transform(LOGICAL_RULES, relNode.getTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
@@ -125,15 +141,35 @@ public class DrillSqlWorker {
       convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
     }
     
-    System.out.println(RelOptUtil.toString(convertedRelNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    //Debug
+    if(logger.isDebugEnabled()) {
+      logger.debug("Drill LogicalRel tree : " + RelOptUtil.toString(convertedRelNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    }
     
     return new RelResult(resultMode, convertedRelNode);
   }
   
-  
-  
+  /*
+   * Given a Drill LogicalRel tree, return Drill Logical Plan. 
+   * @param relResult :  RelResult whose node is the root of Drill logicalrel tree.
+   */
+  public LogicalPlan getLogicalPlan(RelResult relResult) throws SqlParseException, ValidationException, RelConversionException{
+    RelNode logicalRelRoot = relResult.node;
+    
+    Preconditions.checkArgument(logicalRelRoot.getConvention() == DrillRel.DRILL_LOGICAL);
+    
+    DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), relResult.mode);
+    implementor.go( (DrillRel) logicalRelRoot);
+    planner.close();
+    planner.reset();
+    return implementor.getPlan();    
+  }
+
+  /*
+   * Given a SQL string, return the Drill logical plan.
+   */
   public LogicalPlan getLogicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
-    RelResult result = getRel(sql);
+    RelResult result = getLogicalRel(sql);
 
     RelNode convertedRelNode = planner.transform(LOGICAL_RULES, result.node.getTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
     if(convertedRelNode instanceof DrillStoreRel){
@@ -145,22 +181,51 @@ public class DrillSqlWorker {
     implementor.go( (DrillRel) convertedRelNode);
     planner.close();
     planner.reset();
-    return implementor.getPlan();
+    return implementor.getPlan();    
+  }
+
+  /*
+   * Given a Drill LogicalRel tree, return Drill Physical plan.
+   * @param relResult : RelResult whose node is the root of Drill logicalrel tree.
+   * @param qcontext  : QueryContext used by PhysicalPlanCreator. 
+   */
+  public PhysicalPlan getPhysicalPlan(RelResult relResult, QueryContext qcontext) throws SqlParseException, ValidationException, RelConversionException, IOException {
+    RelNode logicalRelRoot = relResult.node;
+    
+    Preconditions.checkArgument(logicalRelRoot.getConvention() == DrillRel.DRILL_LOGICAL);
+    
+    RelTraitSet traits = logicalRelRoot.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);    
+    Prel phyRelNode = (Prel) planner.transform(PHYSICAL_MEM_RULES, traits, logicalRelRoot);
+    
+    //Debug
+    if(logger.isDebugEnabled()) {     
+      String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
+      logger.debug("Drill PhysicalRel tree: " + msg);
+    }
     
+    PhysicalPlanCreator pplanCreator = new PhysicalPlanCreator(qcontext);
+    PhysicalPlan plan = pplanCreator.build(phyRelNode, true /* rebuild */);
+        
+    planner.close();
+    planner.reset();
+    return plan;
   }
   
+  /*
+   * Given a SQL string, return Drill physical plan. 
+   */
   public PhysicalPlan getPhysicalPlan(String sql, QueryContext qcontext) throws SqlParseException, ValidationException, RelConversionException, IOException {
-    RelResult result = getRel(sql);
+    RelResult result = getLogicalRel(sql);
 
     RelTraitSet traits = result.node.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);    
     Prel phyRelNode = (Prel) planner.transform(PHYSICAL_MEM_RULES, traits, result.node);
     
     //Debug.
-    System.err.println("SQL : " + sql);
-    logger.debug("SQL : " + sql);
-    String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
-    System.out.println(msg);
-    logger.debug(msg);
+    if(logger.isDebugEnabled()) {     
+      logger.debug("SQL : " + sql);
+      String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
+      logger.debug("Drill PhysicalRel tree: " + msg);      
+    }
         
     PhysicalPlanCreator pplanCreator = new PhysicalPlanCreator(qcontext);
     PhysicalPlan plan = pplanCreator.build(phyRelNode, true /* rebuild */);
@@ -171,20 +236,4 @@ public class DrillSqlWorker {
 
   }
  
-  public void runPhysicalPlan(PhysicalPlan phyPlan, DrillConfig config) {
-    QuerySubmitter qs = new QuerySubmitter();
-    
-    ObjectMapper mapper = config.getMapper();
-    
-    try {
-      String phyPlanStr = mapper.writeValueAsString(phyPlan);
-      
-      System.out.println(phyPlanStr);
-      
-      qs.submitQuery(null, phyPlanStr, "physical", null, true, 1, "csv");
-    } catch (Exception e) {
-      System.err.println("Query fails " + e.toString());
-    }
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4f98a4f4/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 858508b..4dfb309 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -47,6 +47,7 @@ import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
 import org.apache.drill.exec.planner.fragment.StatsCollector;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
+import org.apache.drill.exec.planner.sql.DrillSqlWorker.RelResult;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
@@ -348,11 +349,24 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     try{
       DrillSqlWorker sqlWorker = new DrillSqlWorker(context.getFactory(), context.getFunctionRegistry());
       
-      PhysicalPlan physical = sqlWorker.getPhysicalPlan(sql, context);
+      RelResult relResult = sqlWorker.getLogicalRel(sql);
       
+      //EXPLAIN logical
+      if (relResult.getMode() == ResultMode.LOGICAL) {
+        returnLogical(sqlWorker.getLogicalPlan(relResult));
+        return;
+      }
+      
+      PhysicalPlan physical = sqlWorker.getPhysicalPlan(relResult, context);
+            
       if(logger.isDebugEnabled()) {
         logger.debug("Distributed Physical {}", context.getConfig().getMapper().writeValueAsString(physical));
-        System.out.println(context.getConfig().getMapper().writeValueAsString(physical));
+      }
+      
+      //EXPLAIN physical
+      if (relResult.getMode() == ResultMode.PHYSICAL) {
+        returnPhysical(physical);
+        return;
       }
       
       runPhysicalPlan(physical);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4f98a4f4/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 5baaf63..35c4707 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
@@ -56,6 +56,16 @@ public class TestExampleQueries {
   public void testGroupBy() throws Exception{
     test("select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
   }
+
+  @Test
+  public void testExpalinPhysical() throws Exception{
+    test("explain plan for select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
+  }
+
+  @Test
+  public void testExpalinLogical() throws Exception{
+    test("explain plan without implementation for select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
+  }
   
   private void test(String sql) throws Exception{
     boolean good = false;


[24/51] [abbrv] git commit: allocate 100 bytes/record for variable length in partition sender

Posted by ja...@apache.org.
allocate 100 bytes/record for variable length in partition sender


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

Branch: refs/heads/master
Commit: f48d9988f9d77d95fbc89fbdf5e8c06672948568
Parents: f2f889c
Author: Steven Phillips <sp...@maprtech.com>
Authored: Tue Apr 1 11:45:04 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../exec/physical/impl/partitionsender/OutgoingRecordBatch.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f48d9988/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index 60b395f..cd7e632 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -171,7 +171,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
 
       // allocate a new value vector
       ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), context.getAllocator());
-      VectorAllocator.getAllocator(v.getValueVector(), outgoingVector).alloc(recordCapacity);
+      VectorAllocator.getAllocator(outgoingVector, 100).alloc(recordCapacity);
       vectorContainer.add(outgoingVector);
       logger.debug("Reallocating to cap " + recordCapacity + " because of newly init'd vector : " + v.getValueVector());
     }


[43/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
index aba9ab7..0ea00f1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserServer.java
@@ -17,8 +17,9 @@
  */
 package org.apache.drill.exec.rpc.user;
 
+import java.io.IOException;
+
 import io.netty.buffer.ByteBuf;
-import io.netty.buffer.ByteBufAllocator;
 import io.netty.buffer.ByteBufInputStream;
 import io.netty.channel.Channel;
 import io.netty.channel.EventLoopGroup;
@@ -26,18 +27,19 @@ import io.netty.channel.EventLoopGroup;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.materialize.QueryWritableBatch;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
 import org.apache.drill.exec.proto.UserProtos.BitToUserHandshake;
 import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RpcType;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.proto.UserProtos.UserToBitHandshake;
+import org.apache.drill.exec.rpc.Acks;
 import org.apache.drill.exec.rpc.BasicServer;
 import org.apache.drill.exec.rpc.ProtobufLengthDecoder;
 import org.apache.drill.exec.rpc.RemoteConnection;
 import org.apache.drill.exec.rpc.Response;
 import org.apache.drill.exec.rpc.RpcException;
 import org.apache.drill.exec.rpc.RpcOutcomeListener;
-import org.apache.drill.exec.rpc.data.DataProtobufLengthDecoder;
 import org.apache.drill.exec.work.user.UserWorker;
 
 import com.google.protobuf.InvalidProtocolBufferException;
@@ -93,21 +95,35 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
         throw new RpcException("Failure while decoding RequestResults body.", e);
       }
 
+    case RpcType.CANCEL_QUERY_VALUE:
+      logger.warn("Cancel requested but not supported yet.");
+      return new Response(RpcType.ACK, Acks.OK);
+
     default:
       throw new UnsupportedOperationException(String.format("UserServer received rpc of unknown type.  Type was %d.", rpcType));
     }
 
   }
 
-  
+
   public class UserClientConnection extends RemoteConnection {
+
+    private UserSession session;
     public UserClientConnection(Channel channel) {
       super(channel);
     }
 
+    void setUser(UserCredentials credentials) throws IOException{
+      session = new UserSession(this, credentials, worker.getSchemaFactory());
+    }
+
+    public UserSession getSession(){
+      return session;
+    }
+
     public void sendResult(RpcOutcomeListener<Ack> listener, QueryWritableBatch result){
 //      logger.debug("Sending result to client with {}", result);
-      
+
       send(listener, this, RpcType.QUERY_RESULT, result.getHeader(), Ack.class, result.getBuffers());
     }
 
@@ -122,15 +138,16 @@ public class UserServer extends BasicServer<RpcType, UserServer.UserClientConnec
   public UserClientConnection initRemoteConnection(Channel channel) {
     return new UserClientConnection(channel);
   }
-  
+
   @Override
-  protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler(UserClientConnection connection) {
+  protected ServerHandshakeHandler<UserToBitHandshake> getHandshakeHandler(final UserClientConnection connection) {
     return new ServerHandshakeHandler<UserToBitHandshake>(RpcType.HANDSHAKE, UserToBitHandshake.PARSER){
 
       @Override
       public MessageLite getHandshakeResponse(UserToBitHandshake inbound) throws Exception {
 //        logger.debug("Handling handshake from user to bit. {}", inbound);
         if(inbound.getRpcVersion() != UserRpcConfig.RPC_VERSION) throw new RpcException(String.format("Invalid rpc version.  Expected %d, actual %d.", inbound.getRpcVersion(), UserRpcConfig.RPC_VERSION));
+        connection.setUser(inbound.getCredentials());
         return BitToUserHandshake.newBuilder().setRpcVersion(UserRpcConfig.RPC_VERSION).build();
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.java
new file mode 100644
index 0000000..75545ec
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/UserSession.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.rpc.user;
+
+import java.io.IOException;
+
+import net.hydromatic.optiq.SchemaPlus;
+
+import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.store.SchemaFactory;
+
+public class UserSession {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserSession.class);
+
+  private DrillUser user;
+  private String defaultSchema = "";
+  private boolean enableExchanges = true;
+  private UserClientConnection connection;
+
+  public UserSession(UserClientConnection connection, UserCredentials credentials, SchemaFactory factory) throws IOException{
+    this.connection = connection;
+  }
+
+
+  public DrillUser getUser(){
+    return user;
+  }
+
+
+  /**
+   * Update the schema path for the session.
+   * @param fullPath The desired path to set to.
+   * @param schema The root schema to find this path within.
+   * @return true if the path was set succesfully.  false if this path was unavailable.
+   */
+  public boolean setDefaultSchemaPath(String fullPath, SchemaPlus schema){
+    SchemaPlus newDefault = getDefaultSchema(schema);
+    if(newDefault == null) return false;
+    this.defaultSchema = fullPath;
+    return true;
+  }
+
+  public SchemaPlus getDefaultSchema(SchemaPlus rootSchema){
+    String[] paths = defaultSchema.split("\\.");
+    SchemaPlus schema = rootSchema;
+    for(String p : paths){
+      schema = schema.getSubSchema(p);
+      if(schema == null) break;
+    }
+    return schema;
+  }
+
+  public void enableExchanges(boolean enableExchanges){
+    this.enableExchanges = enableExchanges;
+  }
+
+  public boolean isEnableExchanges() {
+    return enableExchanges;
+  }
+
+  public boolean setSessionOption(String name, String value){
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
index 159a6f4..14049ef 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/schema/Field.java
@@ -41,11 +41,12 @@ public abstract class Field {
 
   public MaterializedField getAsMaterializedField(FieldReference ref) {
     assert (ref!=null);
-    return MaterializedField.create(new SchemaPath(ref.getPath() + "." + getFieldName(), ExpressionPosition.UNKNOWN), fieldType);  
+
+    return MaterializedField.create(ref.getChild(getFieldName()), fieldType);
   }
 
   public MaterializedField getAsMaterializedField() {
-      return MaterializedField.create(new SchemaPath(getFieldName(), ExpressionPosition.UNKNOWN), fieldType);    
+      return MaterializedField.create(SchemaPath.getCompoundPath(getFieldName().split("\\.")), fieldType);
   }
 
   public abstract String getFieldName();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
index 8edbcc5..1fd30e3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/server/DrillbitContext.java
@@ -43,7 +43,7 @@ import com.google.common.base.Preconditions;
 public class DrillbitContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillbitContext.class);
 
-  private BootStrapContext context;
+  private final BootStrapContext context;
 
   private PhysicalPlanReader reader;
   private final ClusterCoordinator coord;
@@ -55,7 +55,7 @@ public class DrillbitContext {
   private final Controller controller;
   private final WorkEventBus workBus;
   private final FunctionImplementationRegistry functionRegistry;
-  
+
   public DrillbitContext(DrillbitEndpoint endpoint, BootStrapContext context, ClusterCoordinator coord, Controller controller, DataConnectionCreator connectionsPool, DistributedCache cache, WorkEventBus workBus) {
     super();
     Preconditions.checkNotNull(endpoint);
@@ -74,7 +74,7 @@ public class DrillbitContext {
     this.operatorCreatorRegistry = new OperatorCreatorRegistry(context.getConfig());
     this.functionRegistry = new FunctionImplementationRegistry(context.getConfig());
   }
-  
+
   public FunctionImplementationRegistry getFunctionImplementationRegistry() {
     return functionRegistry;
   }
@@ -82,15 +82,15 @@ public class DrillbitContext {
   public WorkEventBus getWorkBus(){
     return workBus;
   }
-  
+
   public DrillbitEndpoint getEndpoint(){
     return endpoint;
   }
-  
+
   public DrillConfig getConfig() {
     return context.getConfig();
   }
-  
+
   public Collection<DrillbitEndpoint> getBits(){
     return coord.getAvailableEndpoints();
   }
@@ -106,34 +106,34 @@ public class DrillbitContext {
   public StoragePluginRegistry getStorage(){
     return this.storagePlugins;
   }
-  
+
   public NioEventLoopGroup getBitLoopGroup(){
     return context.getBitLoopGroup();
   }
-  
-  
+
+
   public DataConnectionCreator getDataConnectionsPool(){
     return connectionsPool;
   }
-  
+
   public Controller getController(){
     return controller;
   }
-  
+
   public MetricRegistry getMetrics(){
     return context.getMetrics();
   }
-  
+
   public DistributedCache getCache(){
     return cache;
   }
-  
+
   public PhysicalPlanReader getPlanReader(){
     return reader;
   }
-  
+
   public DrillSchemaFactory getSchemaFactory(){
     return storagePlugins.getSchemaFactory();
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
index a8d053c..60ccbe8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/SchemaFactory.java
@@ -17,11 +17,12 @@
  */
 package org.apache.drill.exec.store;
 
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
+import org.apache.drill.exec.rpc.user.DrillUser;
+
 public interface SchemaFactory {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SchemaFactory.class);
-  
-  public Schema add(SchemaPlus parent);
+
+  public void registerSchemas(DrillUser user, SchemaPlus parent);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 1c986a9..67f6df3 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
@@ -20,14 +20,14 @@ package org.apache.drill.exec.store;
 import java.io.IOException;
 import java.util.List;
 
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.rpc.user.DrillUser;
 
-public interface StoragePlugin {
+public interface StoragePlugin extends SchemaFactory{
   public boolean supportsRead();
 
   public boolean supportsWrite();
@@ -36,15 +36,14 @@ public interface StoragePlugin {
 
   /**
    * Get the physical scan operator for the particular GroupScan (read) node.
-   * 
+   *
    * @param scan
    *          The configured scan with a storage engine specific selection.
    * @return
    * @throws IOException
    */
   public AbstractGroupScan getPhysicalScan(JSONOptions selection) throws IOException;
-  
-  public Schema createAndAddSchema(SchemaPlus parent);
+
   public StoragePluginConfig getConfig();
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 3594028..2386915 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
@@ -21,20 +21,15 @@ import java.io.IOException;
 import java.lang.reflect.Constructor;
 import java.lang.reflect.InvocationTargetException;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Set;
 
 import net.hydromatic.linq4j.expressions.DefaultExpression;
 import net.hydromatic.linq4j.expressions.Expression;
-import net.hydromatic.linq4j.function.Function1;
-import net.hydromatic.optiq.Function;
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.Table;
+import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -43,6 +38,7 @@ import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.util.PathScanner;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.planner.logical.StorageEngines;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.apache.drill.exec.store.dfs.FormatPlugin;
@@ -51,7 +47,6 @@ import org.apache.drill.exec.store.ischema.InfoSchemaStoragePlugin;
 
 import com.google.common.base.Charsets;
 import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Maps;
 import com.google.common.io.Resources;
 
 
@@ -63,9 +58,9 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
 
   private DrillbitContext context;
   private final DrillSchemaFactory schemaFactory = new DrillSchemaFactory();
-  
+
   private static final Expression EXPRESSION = new DefaultExpression(Object.class);
-  
+
   public StoragePluginRegistry(DrillbitContext context) {
     try{
     this.context = context;
@@ -96,10 +91,10 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
         logger.debug("Skipping registration of StorageEngine {} as it doesn't have a constructor with the parameters of (StorangeEngineConfig, Config)", engine.getCanonicalName());
       }
     }
-    
-    
+
+
   }
-  
+
   private Map<String, StoragePlugin> createEngines(){
     StorageEngines engines = null;
     Map<String, StoragePlugin> activeEngines = new HashMap<String, StoragePlugin>();
@@ -109,7 +104,7 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }catch(IOException e){
       throw new IllegalStateException("Failure while reading storage engines data.", e);
     }
-    
+
     for(Map.Entry<String, StoragePluginConfig> config : engines){
       try{
         StoragePlugin plugin = create(config.getKey(), config.getValue());
@@ -119,14 +114,14 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
       }
     }
     activeEngines.put("INFORMATION_SCHEMA", new InfoSchemaStoragePlugin(new InfoSchemaConfig(), context, "INFORMATION_SCHEMA"));
-    
+
     return activeEngines;
   }
 
   public StoragePlugin getEngine(String registeredStorageEngineName) throws ExecutionSetupException {
     return engines.get(registeredStorageEngineName);
   }
-  
+
   public StoragePlugin getEngine(StoragePluginConfig config) throws ExecutionSetupException {
     if(config instanceof NamedStoragePluginConfig){
       return engines.get(((NamedStoragePluginConfig) config).name);
@@ -135,7 +130,7 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
       return create(null, config);
     }
   }
-  
+
   public FormatPlugin getFormatPlugin(StoragePluginConfig storageConfig, FormatPluginConfig formatConfig) throws ExecutionSetupException{
     StoragePlugin p = getEngine(storageConfig);
     if(!(p instanceof FileSystemPlugin)) throw new ExecutionSetupException(String.format("You tried to request a format plugin for a stroage engine that wasn't of type FileSystemPlugin.  The actual type of plugin was %s.", p.getClass().getName()));
@@ -165,147 +160,20 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
   public Iterator<Entry<String, StoragePlugin>> iterator() {
     return engines.entrySet().iterator();
   }
-  
+
   public DrillSchemaFactory getSchemaFactory(){
     return schemaFactory;
   }
 
-  public class DrillSchemaFactory implements Function1<SchemaPlus, Schema>{
+  public class DrillSchemaFactory implements SchemaFactory{
 
     @Override
-    public Schema apply(SchemaPlus parent) {
-      Schema defaultSchema = null;
+    public void registerSchemas(DrillUser user, SchemaPlus parent) {
       for(Map.Entry<String, StoragePlugin> e : engines.entrySet()){
-        Schema s = e.getValue().createAndAddSchema(parent);
-        if(defaultSchema == null) defaultSchema = s;
+        e.getValue().registerSchemas(user, parent);
       }
-      return defaultSchema;
-    }
-    
-    /**
-     * Used in situations where we want to get a schema without having to use in the context of an Optiq planner. 
-     * @return Root schema of the storage engine hiearchy.
-     */
-    public SchemaPlus getOrphanedRootSchema(){
-      SchemaPlus p = new OrphanPlus();
-      apply(p);
-      return p;
-    }
-    
-  }
-  
-
-  private class OrphanPlusWrap extends OrphanPlus{
-    private Schema inner;
-
-    public OrphanPlusWrap(Schema inner) {
-      super();
-      this.inner = inner;
     }
 
-    @Override
-    public Table getTable(String name) {
-      return inner.getTable(name);
-    }
-
-    @Override
-    public Set<String> getTableNames() {
-      return inner.getTableNames();
-    }
-
-    @Override
-    public Collection<Function> getFunctions(String name) {
-      return inner.getFunctions(name);
-    }
-
-    @Override
-    public Set<String> getFunctionNames() {
-      return inner.getFunctionNames();
-    }
-
-    @Override
-    public Set<String> getSubSchemaNames() {
-      return inner.getSubSchemaNames();
-    }
-    
   }
 
-  private class OrphanPlus implements SchemaPlus{
-
-    private HashMap<String, SchemaPlus> schemas = Maps.newHashMap();
-    
-    @Override
-    public SchemaPlus getParentSchema() {
-      return null;
-    }
-
-    @Override
-    public String getName() {
-      return "";
-    }
-
-    @Override
-    public Table getTable(String name) {
-      return null;
-    }
-
-    @Override
-    public Set<String> getTableNames() {
-      return Collections.emptySet();
-    }
-
-    @Override
-    public Collection<Function> getFunctions(String name) {
-      return Collections.emptyList();
-    }
-
-    @Override
-    public Set<String> getFunctionNames() {
-      return Collections.emptySet();
-    }
-
-    @Override
-    public Set<String> getSubSchemaNames() {
-      return schemas.keySet();
-    }
-
-    @Override
-    public SchemaPlus getSubSchema(String name) {
-      return schemas.get(name);
-    }
-
-    @Override
-    public void add(String name, Table table) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void add(String name, Function table) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public boolean isMutable() {
-      return false;
-    }
-
-    @Override
-    public <T> T unwrap(Class<T> clazz) {
-      throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public Expression getExpression(SchemaPlus parentSchema, String name) {
-      return EXPRESSION;
-    }
-
-    @Override
-    public SchemaPlus add(String name, Schema schema) {
-      OrphanPlusWrap plus = new OrphanPlusWrap(schema);
-      schemas.put(name, plus);
-      return plus;
-    }
-
-  }
-  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
index 120fdf4..ebd8507 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemPlugin.java
@@ -29,7 +29,10 @@ import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.logical.FormatPluginConfig;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.rpc.user.DrillUser;
+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.ClassPathFileSystem;
@@ -39,6 +42,7 @@ import org.apache.hadoop.conf.Configuration;
 
 import com.beust.jcommander.internal.Lists;
 import com.beust.jcommander.internal.Maps;
+
 import org.apache.hadoop.fs.FileSystem;
 
 /**
@@ -56,12 +60,12 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
   private FileSystemConfig config;
   private DrillbitContext context;
   private final DrillFileSystem fs;
-  
+
   public FileSystemPlugin(FileSystemConfig config, DrillbitContext context, String name) throws ExecutionSetupException{
     try{
       this.config = config;
       this.context = context;
-      
+
       Configuration fsConf = new Configuration();
       fsConf.set(FileSystem.FS_DEFAULT_NAME_KEY, config.connection);
       fsConf.set("fs.classpath.impl", ClassPathFileSystem.class.getName());
@@ -73,7 +77,7 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
         matchers.add(p.getMatcher());
         formatPluginsByConfig.put(p.getConfig(), p);
       }
-      
+
       List<WorkspaceSchemaFactory> factories = null;
       if(config.workspaces == null || config.workspaces.isEmpty()){
         factories = Collections.singletonList(new WorkspaceSchemaFactory(this, "default", name, fs, "/", matchers));
@@ -88,12 +92,12 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
       throw new ExecutionSetupException("Failure setting up file system plugin.", e);
     }
   }
-  
+
   @Override
   public boolean supportsRead() {
     return true;
   }
-  
+
   @Override
   public StoragePluginConfig getConfig() {
     return config;
@@ -111,16 +115,16 @@ public class FileSystemPlugin extends AbstractStoragePlugin{
     if(plugin == null) throw new IOException(String.format("Failure getting requested format plugin named '%s'.  It was not one of the format plugins registered.", formatSelection.getFormat()));
     return plugin.getGroupScan(formatSelection.getSelection());
   }
-  
+
   @Override
-  public Schema createAndAddSchema(SchemaPlus parent) {
-    return schemaFactory.add(parent);
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
+    schemaFactory.registerSchemas(user, parent);
   }
-  
+
   public FormatPlugin getFormatPlugin(String name){
     return formatsByName.get(name);
   }
-  
+
   public FormatPlugin getFormatPlugin(FormatPluginConfig config){
     if(config instanceof NamedFormatPluginConfig){
       return formatsByName.get(((NamedFormatPluginConfig) config).name);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index f569098..b3f240e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -27,6 +27,7 @@ import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory.WorkspaceSchema;
@@ -42,8 +43,8 @@ public class FileSystemSchemaFactory implements SchemaFactory{
 
   private List<WorkspaceSchemaFactory> factories;
   private String schemaName;
-  
-  
+
+
   public FileSystemSchemaFactory(String schemaName, List<WorkspaceSchemaFactory> factories) {
     super();
     this.schemaName = schemaName;
@@ -51,25 +52,24 @@ public class FileSystemSchemaFactory implements SchemaFactory{
   }
 
   @Override
-  public Schema add(SchemaPlus parent) {
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
     FileSystemSchema schema = new FileSystemSchema(schemaName);
     SchemaPlus plusOfThis = parent.add(schema.getName(), schema);
     schema.setPlus(plusOfThis);
-    return schema;
   }
 
   public class FileSystemSchema extends AbstractSchema{
 
     private final WorkspaceSchema defaultSchema;
     private final Map<String, WorkspaceSchema> schemaMap = Maps.newHashMap();
-    
+
     public FileSystemSchema(String name) {
       super(name);
       for(WorkspaceSchemaFactory f :  factories){
         WorkspaceSchema s = f.createSchema();
         schemaMap.put(s.getName(), s);
       }
-      
+
       defaultSchema = schemaMap.get("default");
     }
 
@@ -78,7 +78,7 @@ public class FileSystemSchemaFactory implements SchemaFactory{
         plusOfThis.add(s.getName(), s);
       }
     }
-    
+
     @Override
     public DrillTable getTable(String name) {
       return defaultSchema.getTable(name);
@@ -108,7 +108,7 @@ public class FileSystemSchemaFactory implements SchemaFactory{
     public Set<String> getTableNames() {
       return defaultSchema.getTableNames();
     }
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectBatchCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectBatchCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectBatchCreator.java
new file mode 100644
index 0000000..fb16edf
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectBatchCreator.java
@@ -0,0 +1,37 @@
+/**
+ * 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.direct;
+
+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.RecordBatch;
+
+public class DirectBatchCreator implements BatchCreator<DirectSubScan>{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectBatchCreator.class);
+
+  @Override
+  public RecordBatch getBatch(FragmentContext context, DirectSubScan config, List<RecordBatch> children)
+      throws ExecutionSetupException {
+    return new ScanBatch(context, Collections.singleton(config.getReader()).iterator());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
new file mode 100644
index 0000000..cfbeb83
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectGroupScan.java
@@ -0,0 +1,81 @@
+/**
+ * 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.direct;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.physical.EndpointAffinity;
+import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.physical.base.SubScan;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.RecordReader;
+
+public class DirectGroupScan extends AbstractGroupScan{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectGroupScan.class);
+
+  private final RecordReader reader;
+
+  public DirectGroupScan(RecordReader reader) {
+    super();
+    this.reader = reader;
+  }
+
+  @Override
+  public void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException {
+    assert endpoints.size() == 1;
+  }
+
+  @Override
+  public SubScan getSpecificScan(int minorFragmentId) throws ExecutionSetupException {
+    assert minorFragmentId == 0;
+    return new DirectSubScan(reader);
+  }
+
+  @Override
+  public int getMaxParallelizationWidth() {
+    return 1;
+  }
+
+  @Override
+  public OperatorCost getCost() {
+    return new OperatorCost(1,1,1,1);
+  }
+
+  @Override
+  public Size getSize() {
+    return new Size(1,1);
+  }
+
+  @Override
+  public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) throws ExecutionSetupException {
+    assert children == null || children.isEmpty();
+    return this;
+  }
+
+  @Override
+  public List<EndpointAffinity> getOperatorAffinity() {
+    return Collections.emptyList();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
new file mode 100644
index 0000000..0c50898
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/direct/DirectSubScan.java
@@ -0,0 +1,37 @@
+/**
+ * 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.direct;
+
+import org.apache.drill.exec.physical.base.AbstractSubScan;
+import org.apache.drill.exec.store.RecordReader;
+
+public class DirectSubScan extends AbstractSubScan{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectSubScan.class);
+
+  private final RecordReader reader;
+
+  public DirectSubScan(RecordReader reader) {
+    super();
+    this.reader = reader;
+  }
+
+  public RecordReader getReader() {
+    return reader;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 8d502a1..fb23a09 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
@@ -92,7 +92,7 @@ public class JSONRecordReader implements RecordReader {
     this.columns = columns;
   }
 
-  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, FileSystem fileSystem, 
+  public JSONRecordReader(FragmentContext fragmentContext, String inputPath, FileSystem fileSystem,
                           List<SchemaPath> columns) {
     this(fragmentContext, inputPath, fileSystem, DEFAULT_LENGTH, columns);
   }
@@ -204,7 +204,8 @@ public class JSONRecordReader implements RecordReader {
   }
 
   private boolean fieldSelected(String field){
-    SchemaPath sp = new SchemaPath(field, ExpressionPosition.UNKNOWN);
+
+    SchemaPath sp = SchemaPath.getCompoundPath(field.split("\\."));
     if (this.columns != null && this.columns.size() > 0){
       for (SchemaPath expr : this.columns){
         if ( sp.equals(expr)){
@@ -519,7 +520,7 @@ public class JSONRecordReader implements RecordReader {
         return null;
       }
 
-      MaterializedField f = MaterializedField.create(new SchemaPath(fullFieldName, ExpressionPosition.UNKNOWN), type);
+      MaterializedField f = MaterializedField.create(SchemaPath.getCompoundPath(fullFieldName.split("\\.")), type);
 
       ValueVector v = TypeHelper.getNewVector(f, allocator);
       AllocationHelper.allocate(v, batchSize, 50);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
index 5bd520d..c659b1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveRecordReader.java
@@ -139,7 +139,7 @@ public class HiveRecordReader implements RecordReader {
         List<Integer> columnIds = Lists.newArrayList();
         columnNames = Lists.newArrayList();
         for (FieldReference field : columns) {
-          String columnName = field.getPath().toString();
+          String columnName = field.getRootSegment().getPath();
           if (!tableColumns.contains(columnName)) {
             if (partition != null && partitionNames.contains(columnName)) {
               selectedPartitionNames.add(columnName);
@@ -195,14 +195,14 @@ public class HiveRecordReader implements RecordReader {
     try {
       for (int i = 0; i < columnNames.size(); i++) {
         PrimitiveCategory pCat = primitiveCategories.get(i);
-        MaterializedField field = MaterializedField.create(new SchemaPath(columnNames.get(i), ExpressionPosition.UNKNOWN), getMajorType(pCat));
+        MaterializedField field = MaterializedField.create(SchemaPath.getSimplePath(columnNames.get(i)), getMajorType(pCat));
         ValueVector vv = TypeHelper.getNewVector(field, context.getAllocator());
         vectors.add(vv);
         output.addField(vv);
       }
       for (int i = 0; i < selectedPartitionNames.size(); i++) {
         String type = selectedPartitionTypes.get(i);
-        MaterializedField field = MaterializedField.create(new SchemaPath(selectedPartitionNames.get(i), ExpressionPosition.UNKNOWN), getMajorType(type));
+        MaterializedField field = MaterializedField.create(SchemaPath.getSimplePath(columnNames.get(i)), getMajorType(type));
         ValueVector vv = TypeHelper.getNewVector(field, context.getAllocator());
         pVectors.add(vv);
         output.addField(vv);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
index 46d85a5..e6df669 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/HiveStoragePlugin.java
@@ -24,6 +24,7 @@ import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.hive.schema.HiveSchemaFactory;
@@ -35,7 +36,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
 public class HiveStoragePlugin extends AbstractStoragePlugin {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HiveStoragePlugin.class);
-  
+
   private final HiveStoragePluginConfig config;
   private final HiveSchemaFactory schemaFactory;
   private final DrillbitContext context;
@@ -55,7 +56,7 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
   public String getName(){
     return name;
   }
-  
+
   public DrillbitContext getContext() {
     return context;
   }
@@ -71,8 +72,8 @@ public class HiveStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public Schema createAndAddSchema(SchemaPlus parent) {
-    return schemaFactory.add(parent);
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
+    schemaFactory.registerSchemas(user, parent);
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
index 29eda92..7b30a41 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/hive/schema/HiveSchemaFactory.java
@@ -28,6 +28,7 @@ import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.store.hive.HiveReadEntry;
@@ -62,7 +63,7 @@ public class HiveSchemaFactory implements SchemaFactory {
   public HiveSchemaFactory(HiveStoragePlugin plugin, String name, HiveConf hiveConf) throws ExecutionSetupException {
     this.schemaName = name;
     this.plugin = plugin;
-    
+
     try {
       this.mClient = new HiveMetaStoreClient(hiveConf);
     } catch (MetaException e) {
@@ -170,22 +171,21 @@ public class HiveSchemaFactory implements SchemaFactory {
   }
 
   @Override
-  public Schema add(SchemaPlus parent) {
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
     HiveSchema schema = new HiveSchema(schemaName);
     SchemaPlus hPlus = parent.add(schemaName, schema);
     schema.setHolder(hPlus);
-    return schema;
   }
 
   class HiveSchema extends AbstractSchema {
 
     private HiveDatabaseSchema defaultSchema;
-    
+
     public HiveSchema(String name) {
       super(name);
       getSubSchema("default");
     }
-    
+
     @Override
     public Schema getSubSchema(String name) {
       List<String> tables;
@@ -200,16 +200,16 @@ public class HiveSchemaFactory implements SchemaFactory {
         logger.warn("Failure while attempting to access HiveDatabase '{}'.", name, e.getCause());
         return null;
       }
-      
+
     }
-    
+
 
     void setHolder(SchemaPlus plusOfThis){
       for(String s : getSubSchemaNames()){
         plusOfThis.add(s, getSubSchema(s));
       }
     }
-    
+
 
     @Override
     public Set<String> getSubSchemaNames() {
@@ -224,7 +224,7 @@ public class HiveSchemaFactory implements SchemaFactory {
 
     @Override
     public DrillTable getTable(String name) {
-      if(defaultSchema == null){ 
+      if(defaultSchema == null){
         return super.getTable(name);
       }
       return defaultSchema.getTable(name);
@@ -246,13 +246,13 @@ public class HiveSchemaFactory implements SchemaFactory {
         return Collections.emptyList();
       }
     }
-    
+
     DrillTable getDrillTable(String dbName, String t){
       HiveReadEntry entry = getSelectionBaseOnName(dbName, t);
       if(entry == null) return null;
       return new DrillHiveTable(schemaName, plugin, entry);
     }
-    
+
     HiveReadEntry getSelectionBaseOnName(String dbName, String t) {
       if(dbName == null) dbName = "default";
       try{
@@ -262,7 +262,7 @@ public class HiveSchemaFactory implements SchemaFactory {
         return null;
       }
     }
-    
+
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/EmptyVectorSet.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/EmptyVectorSet.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/EmptyVectorSet.java
index 5379bc9..89c9c1e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/EmptyVectorSet.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/EmptyVectorSet.java
@@ -43,7 +43,7 @@ import org.apache.drill.exec.vector.VarCharVector;
  * handles the generic handling of the vectors.
  */
 public abstract class EmptyVectorSet implements VectorSet {
-  
+
   protected List<ValueVector> vectors;
 
   /**
@@ -61,18 +61,18 @@ public abstract class EmptyVectorSet implements VectorSet {
    */
   @Override
   public void beginBatch(int maxRows) {
-    
+
     // Allocate memory for each column (value vector)
     for (ValueVector v: vectors) {
       AllocationHelper.allocate(v, maxRows, 100); // TODO: later, use configured size
     }
   }
-  
-  
+
+
   /**
-   * Write a row to the value vectors. 
+   * Write a row to the value vectors.
    * This is a routine to "assign generic objects to generic ValueVectors"
-   * which can be overridden to optimize for fixed types of vectors and 
+   * which can be overridden to optimize for fixed types of vectors and
    * fixed types of values.
    * @param index - the position within the value vectors.
    * @param row - the objects to write into the vectors
@@ -84,25 +84,25 @@ public abstract class EmptyVectorSet implements VectorSet {
       if (!setSafe(vectors.get(i), index, row[i])) {
         return false;
       }
-    } 
+    }
     return true;
   }
-  
 
-  
+
+
   /**
    * Signal the end of the current batch.
    * @param actualRowCount
    */
   @Override
   public void endBatch(int actualRowCount) {
-    
+
     // Finalize each of the value vectors.
     for (ValueVector v: vectors) {
       v.getMutator().setValueCount(actualRowCount);
     }
   }
-  
+
   /**
    * When everything is done, free up the resources.
    */
@@ -112,15 +112,15 @@ public abstract class EmptyVectorSet implements VectorSet {
       v.close();
     }
   }
-  
-  
+
+
   /**
    * Make the value vectors visible to whomever needs them.
    */
   public List<ValueVector> getValueVectors() {
     return vectors;
   }
-  
+
 
   /**
    * Estimate how many rows will fit in a given amount of memory.
@@ -133,23 +133,23 @@ public abstract class EmptyVectorSet implements VectorSet {
   }
 
 
- 
+
   /**
    * Estimate the size of an average row. Used for allocating memory.
    * Override when more information is known about the data.
    * @return bytes per row.
    */
   protected int getEstimatedRowSize() {
-    
+
     // Add up the sizes of the vectors
     int size = 0;
     for (ValueVector v: vectors) {
-      size += TypeHelper.getSize(v.getField().getType());  
+      size += TypeHelper.getSize(v.getField().getType());
     }
     return size;
   }
-  
-  
+
+
   /**
    * Helper function to create value vectors for a set of columns.
    * @param names - the names of the fields
@@ -162,8 +162,8 @@ public abstract class EmptyVectorSet implements VectorSet {
       vectors.add(createVector(names[i], types[i], allocator));
     }
   }
-  
- 
+
+
   /**
    * Create a value vector for a single column.
    * @param name - the name of the field
@@ -174,8 +174,8 @@ public abstract class EmptyVectorSet implements VectorSet {
   private static ValueVector createVector(String name, MajorType type, BufferAllocator allocator) {
     return TypeHelper.getNewVector(field(name, type), allocator);
   }
-  
-  
+
+
   /**
    * Helper function to create a MaterializedField, used to create a ValueVector.
    * @param name - the name of the field
@@ -183,10 +183,10 @@ public abstract class EmptyVectorSet implements VectorSet {
    * @return the MaterializedField
    */
   private static MaterializedField field(String name, MajorType majorType) {
-    return MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), majorType);
+    return MaterializedField.create(SchemaPath.getSimplePath(name), majorType);
   }
-  
-  
+
+
   //////////////////////////////////////////////////////////////////
   //
   // The following section contains wrappers around ValueVectors.
@@ -200,14 +200,14 @@ public abstract class EmptyVectorSet implements VectorSet {
   //
   ///////////////////////////////////////////////////////////////////
   static final Charset UTF8 = Charset.forName("UTF-8");
-  
-  
-  // Here are the types used in information schema. 
+
+
+  // Here are the types used in information schema.
   public static final MajorType VARCHAR = Types.required(MinorType.VARCHAR);
   public static final MajorType INT = Types.required(MinorType.INT);
   //public static final MajorType NULLABLINT = Types.optional(MinorType.INT);
-  
-  
+
+
   /**
    * A generic routine to set a Java value into a value vector. It assumes the types are compatible.
    * When a subclass knows the types of its columns, it should use the strongly typed routines instead.
@@ -224,17 +224,17 @@ public abstract class EmptyVectorSet implements VectorSet {
     }
   }
 
-  
+
   /**
-   * Strongly typed routines for setting a Java value into a value vector. 
+   * Strongly typed routines for setting a Java value into a value vector.
    * @return true if the value was successfully set.
    */
   protected static boolean setSafe(VarCharVector v, int index, String string) {
     return v.getMutator().setSafe(index, string.getBytes(UTF8));
-  } 
-  
+  }
+
   protected static boolean setSafe(IntVector v, int index, int value) {
     return v.getMutator().setSafe(index, value);
   }
-   
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
index 9cc17bd..313ea86 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
@@ -21,12 +21,12 @@ import java.io.IOException;
 import java.util.Map;
 import java.util.Set;
 
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.planner.logical.DrillTable;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
@@ -40,13 +40,13 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin{
   private final InfoSchemaConfig config;
   private final DrillbitContext context;
   private final String name;
-  
+
   public InfoSchemaStoragePlugin(InfoSchemaConfig config, DrillbitContext context, String name){
     this.config = config;
     this.context = context;
     this.name = name;
   }
-  
+
   @Override
   public boolean supportsRead() {
     return true;
@@ -62,34 +62,33 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin{
   public StoragePluginConfig getConfig() {
     return this.config;
   }
-  
+
   @Override
-  public Schema createAndAddSchema(SchemaPlus parent) {
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
     ISchema s = new ISchema(parent, this);
     parent.add(s.getName(), s);
-    return s;
   }
-  
+
   private class ISchema extends AbstractSchema{
     private Map<String, InfoSchemaDrillTable> tables;
     public ISchema(SchemaPlus parent, InfoSchemaStoragePlugin plugin){
       super("INFORMATION_SCHEMA");
       Map<String, InfoSchemaDrillTable> tbls = Maps.newHashMap();
       for(SelectedTable tbl : SelectedTable.values()){
-        tbls.put(tbl.name(), new InfoSchemaDrillTable(plugin, "INFORMATION_SCHEMA", tbl, config));  
+        tbls.put(tbl.name(), new InfoSchemaDrillTable(plugin, "INFORMATION_SCHEMA", tbl, config));
       }
       this.tables = ImmutableMap.copyOf(tbls);
     }
-    
+
     @Override
     public DrillTable getTable(String name) {
       return tables.get(name);
     }
-    
+
     @Override
     public Set<String> getTableNames() {
       return tables.keySet();
     }
-    
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
index bead67b..cb4f3d9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/OptiqProvider.java
@@ -20,10 +20,10 @@ package org.apache.drill.exec.store.ischema;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 import net.hydromatic.optiq.Table;
+import net.hydromatic.optiq.jdbc.JavaTypeFactoryImpl;
 
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.sql.type.SqlTypeFactoryImpl;
 import org.eigenbase.sql.type.SqlTypeName;
 
 /**
@@ -41,7 +41,7 @@ public class OptiqProvider  {
   /**
    * Provide data for TABLES table.
    */
-  static public class Tables extends Abstract { 
+  static public class Tables extends Abstract {
     Tables(SchemaPlus root) {
       super(root);
     }
@@ -70,7 +70,7 @@ public class OptiqProvider  {
     }
   }
 
-  
+
 
   /**
    * Provide data for COLUMNS data.
@@ -86,7 +86,7 @@ public class OptiqProvider  {
       String columnName = field.getName();
       RelDataType type = field.getType();
       SqlTypeName sqlType = type.getSqlTypeName();
-      
+
       int position = field.getIndex();
       String nullable;
       if (type.isNullable()) nullable = "YES";
@@ -151,19 +151,19 @@ public class OptiqProvider  {
     public void generateRows() {
 
       // Scan the root schema for subschema, tables, columns.
-      scanSchema(root); 
+      scanSchema(root);
     }
   }
 
 
 
   /**
-   * An OptiqScanner scans the Optiq schema, generating rows for each 
+   * An OptiqScanner scans the Optiq schema, generating rows for each
    * schema, table or column. It is intended to be subclassed, where the
    * subclass does what it needs when visiting a Optiq schema structure.
    */
   // We would really prefer multiple inheritance from both OptiqScanner and PipeProvider,
-  //   but making one a subclass of the other works for now. 
+  //   but making one a subclass of the other works for now.
   //   TODO: Refactor to avoid subclassing of what should be an unrelated class.
   abstract static class OptiqScanner extends PipeProvider {
 
@@ -187,7 +187,7 @@ public class OptiqProvider  {
     protected void scanSchema(SchemaPlus root) {
       scanSchema(root.getName(), root);
     }
-    
+
     /**
      * Recursively scan the schema, invoking the visitor as appropriate.
      * @param schemaPath - the path to the current schema, so far,
@@ -195,7 +195,7 @@ public class OptiqProvider  {
      * @param visitor - the methods to invoke at each entity in the schema.
      */
     private void scanSchema(String schemaPath, Schema schema) {
-      
+
       // If we have an empty schema path, then don't insert a leading dot.
       String separator;
       if (schemaPath == "") separator = "";
@@ -213,18 +213,18 @@ public class OptiqProvider  {
         for (String tableName: schema.getTableNames()) {
           if(visitTableName(schemaPath, tableName)){
             Table table = schema.getTable(tableName);
-            
+
             // Visit the table, and if requested ...
             if (visitTable(schemaPath,  tableName, table)) {
 
               // ... do for each of the table's fields.
-              RelDataType tableRow = table.getRowType(new SqlTypeFactoryImpl()); // TODO: Is this correct?
+              RelDataType tableRow = table.getRowType(new JavaTypeFactoryImpl()); // TODO: Is this correct?
               for (RelDataTypeField field: tableRow.getFieldList()) {
 
                 // Visit the field.
                 visitField(schemaPath,  tableName, field);
               }
-            }            
+            }
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
index 1371226..7a2ed1b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockRecordReader.java
@@ -57,11 +57,11 @@ public class MockRecordReader implements RecordReader {
 
   private ValueVector getVector(String name, MajorType type, int length) {
     assert context != null : "Context shouldn't be null.";
-    MaterializedField f = MaterializedField.create(new SchemaPath(name, ExpressionPosition.UNKNOWN), type);
+    MaterializedField f = MaterializedField.create(SchemaPath.getSimplePath(name), type);
     ValueVector v;
     v = TypeHelper.getNewVector(f, context.getAllocator());
     AllocationHelper.allocate(v, length, 50, 4);
-    
+
     return v;
 
   }
@@ -88,17 +88,17 @@ public class MockRecordReader implements RecordReader {
   @Override
   public int next() {
     if(recordsRead >= this.config.getRecords()) return 0;
-    
+
     int recordSetSize = Math.min(batchRecordCount, this.config.getRecords() - recordsRead);
-    
+
     recordsRead += recordSetSize;
     for(ValueVector v : valueVectors){
       AllocationHelper.allocate(v, recordSetSize, 50, 10);
-      
+
 //      logger.debug(String.format("MockRecordReader:  Generating %d records of random data for VV of type %s.", recordSetSize, v.getClass().getName()));
       ValueVector.Mutator m = v.getMutator();
       m.generateTestData(recordSetSize);
-      
+
     }
     return recordSetSize;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
index 0d1635f..10f6e08 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/mock/MockStorageEngine.java
@@ -20,12 +20,12 @@ package org.apache.drill.exec.store.mock;
 import java.io.IOException;
 import java.util.ArrayList;
 
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.AbstractGroupScan;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 import org.apache.drill.exec.store.mock.MockGroupScanPOP.MockScanEntry;
@@ -37,7 +37,7 @@ public class MockStorageEngine extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MockStorageEngine.class);
 
   private final MockStorageEngineConfig configuration;
-  
+
   public MockStorageEngine(MockStorageEngineConfig configuration, DrillbitContext context, String name) {
     this.configuration = configuration;
   }
@@ -48,13 +48,12 @@ public class MockStorageEngine extends AbstractStoragePlugin {
     ArrayList<MockScanEntry> readEntries = selection.getListWith(new ObjectMapper(),
         new TypeReference<ArrayList<MockScanEntry>>() {
         });
-    
+
     return new MockGroupScanPOP(null, readEntries);
   }
 
   @Override
-  public Schema createAndAddSchema(SchemaPlus parent) {
-    return null;
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
index a3c57cf..6e17fba 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetRecordReader.java
@@ -57,7 +57,7 @@ class ParquetRecordReader implements RecordReader {
   private static final long DEFAULT_BATCH_LENGTH = 256 * 1024 * NUMBER_OF_VECTORS; // 256kb
   private static final long DEFAULT_BATCH_LENGTH_IN_BITS = DEFAULT_BATCH_LENGTH * 8; // 256kb
   private static final char DEFAULT_RECORDS_TO_READ_IF_NOT_FIXED_WIDTH = 32*1024;
-  
+
   // TODO - should probably find a smarter way to set this, currently 1 megabyte
   private static final int VAR_LEN_FIELD_LENGTH = 1024 * 1024 * 1;
   public static final int PARQUET_PAGE_MAX_SIZE = 1024 * 1024 * 1;
@@ -164,7 +164,7 @@ class ParquetRecordReader implements RecordReader {
 
   @Override
   public void setup(OutputMutator output) throws ExecutionSetupException {
-    
+
     columnStatuses = new ArrayList<>();
     totalRecords = footer.getBlocks().get(rowGroupIndex).getRowCount();
     List<ColumnDescriptor> columns = footer.getFileMetaData().getSchema().getColumns();
@@ -256,7 +256,7 @@ class ParquetRecordReader implements RecordReader {
   }
 
   private SchemaPath toFieldName(String[] paths) {
-    return new SchemaPath(Joiner.on('/').join(paths), ExpressionPosition.UNKNOWN);
+    return SchemaPath.getCompoundPath(paths);
   }
 
   private TypeProtos.DataMode getDataMode(ColumnDescriptor column) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
new file mode 100644
index 0000000..f0d4fb6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/AbstractWriter.java
@@ -0,0 +1,63 @@
+/**
+ * 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.pojo;
+
+import java.lang.reflect.Field;
+
+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.physical.impl.OutputMutator;
+import org.apache.drill.exec.record.MaterializedField;
+import org.apache.drill.exec.vector.AllocationHelper;
+import org.apache.drill.exec.vector.ValueVector;
+
+abstract class AbstractWriter<V extends ValueVector> implements PojoWriter{
+
+  protected final Field field;
+  protected V vector;
+  protected final MajorType type;
+
+  public AbstractWriter(Field field, MajorType type){
+    this.field = field;
+    this.type = type;
+  }
+
+  @Override
+  public void init(OutputMutator output) throws SchemaChangeException {
+    MaterializedField mf = MaterializedField.create(field.getName(), type);
+    @SuppressWarnings("unchecked")
+    Class<V> valueVectorClass = (Class<V>) TypeHelper.getValueVectorClass(type.getMinorType(), type.getMode());
+    this.vector = output.addField(mf, valueVectorClass);
+  }
+
+  @Override
+  public void allocate() {
+    AllocationHelper.allocate(vector, 500, 100);
+  }
+
+  public void setValueCount(int valueCount){
+    vector.getMutator().setValueCount(valueCount);
+  }
+
+  @Override
+  public void cleanup() {
+  }
+
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
new file mode 100644
index 0000000..8dac455
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoRecordReader.java
@@ -0,0 +1,131 @@
+/**
+ * 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.pojo;
+
+import java.lang.reflect.Field;
+import java.util.Iterator;
+
+import org.apache.drill.common.exceptions.ExecutionSetupException;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+import org.apache.drill.exec.store.RecordReader;
+import org.apache.drill.exec.store.pojo.Writers.BitWriter;
+import org.apache.drill.exec.store.pojo.Writers.IntWriter;
+import org.apache.drill.exec.store.pojo.Writers.LongWriter;
+import org.apache.drill.exec.store.pojo.Writers.StringWriter;
+
+public class PojoRecordReader<T> implements RecordReader{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PojoRecordReader.class);
+
+  public final int forJsonIgnore = 1;
+
+  private final Class<T> pojoClass;
+  private final Iterator<T> iterator;
+  private PojoWriter[] writers;
+  private boolean doCurrent;
+  private T currentPojo;
+
+  public PojoRecordReader(Class<T> pojoClass, Iterator<T> iterator){
+    this.pojoClass = pojoClass;
+    this.iterator = iterator;
+  }
+
+  @Override
+  public void setup(OutputMutator output) throws ExecutionSetupException {
+    try{
+      Field[] fields = pojoClass.getFields();
+      writers = new PojoWriter[fields.length];
+      for(int i = 0; i < writers.length; i++){
+        Field f = fields[i];
+        Class<?> type = f.getType();
+
+        if(type == int.class){
+          writers[i] = new IntWriter(f);
+        }else if(type == boolean.class){
+          writers[i] = new BitWriter(f);
+        }else if(type == long.class){
+          writers[i] = new LongWriter(f);
+        }else if(type == String.class){
+          writers[i] = new StringWriter(f);
+        }else{
+          throw new ExecutionSetupException(String.format("PojoRecord reader doesn't yet support conversions from type [%s].", type));
+        }
+        writers[i].init(output);
+      }
+
+      output.setNewSchema();
+    }catch(SchemaChangeException e){
+      throw new ExecutionSetupException("Failure while setting up schema for PojoRecordReader.", e);
+    }
+
+  }
+
+  private void allocate(){
+    for(PojoWriter writer : writers){
+      writer.allocate();
+    }
+  }
+
+  private void setValueCount(int i){
+    for(PojoWriter writer : writers){
+      writer.setValueCount(i);
+    }
+  }
+
+  @Override
+  public int next() {
+    boolean allocated = false;
+
+    try{
+      int i =0;
+      outside:
+      while(doCurrent || iterator.hasNext()){
+        if(doCurrent){
+          doCurrent = false;
+        }else{
+          currentPojo = iterator.next();
+        }
+
+        if(!allocated){
+          allocate();
+          allocated = true;
+        }
+
+        for(PojoWriter writer : writers){
+          if(!writer.writeField(currentPojo, i)){
+            doCurrent = true;
+            if(i == 0) throw new IllegalStateException("Got into a position where we can't write data but the batch is empty.");
+            break outside;
+          };
+        }
+        i++;
+      }
+
+      if(i != 0 ) setValueCount(i);
+      return i;
+    } catch (IllegalArgumentException | IllegalAccessException e) {
+      throw new RuntimeException("Failure while trying to use PojoRecordReader.", e);
+    }
+  }
+
+  @Override
+  public void cleanup() {
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
new file mode 100644
index 0000000..0ffa55c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/PojoWriter.java
@@ -0,0 +1,29 @@
+/**
+ * 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.pojo;
+
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.physical.impl.OutputMutator;
+
+interface PojoWriter{
+  boolean writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException ;
+  void init(OutputMutator output) throws SchemaChangeException;
+  void allocate();
+  void setValueCount(int i);
+  void cleanup();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
new file mode 100644
index 0000000..6910903
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/pojo/Writers.java
@@ -0,0 +1,127 @@
+/**
+ * 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.pojo;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.UnpooledByteBufAllocator;
+
+import java.lang.reflect.Field;
+
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
+import org.apache.drill.exec.expr.holders.NullableVarCharHolder;
+import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.BitVector;
+import org.apache.drill.exec.vector.IntVector;
+import org.apache.drill.exec.vector.NullableVarCharVector;
+
+import com.google.common.base.Charsets;
+
+public class Writers {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Writers.class);
+
+  public static class IntWriter extends AbstractWriter<IntVector>{
+
+    public IntWriter(Field field) {
+      super(field, Types.required(MinorType.INT));
+      if(field.getType() != int.class) throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
+      int i = field.getInt(pojo);
+      return vector.getMutator().setSafe(outboundIndex, i);
+    }
+
+  }
+
+  public static class BitWriter extends AbstractWriter<BitVector>{
+
+    public BitWriter(Field field) {
+      super(field, Types.required(MinorType.BIT));
+      if(field.getType() != boolean.class) throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
+      boolean b = field.getBoolean(pojo);
+      return vector.getMutator().setSafe(outboundIndex, b ? 1 : 0);
+    }
+
+  }
+
+  public static class LongWriter extends AbstractWriter<BigIntVector>{
+
+    public LongWriter(Field field) {
+      super(field, Types.required(MinorType.BIGINT));
+      if(field.getType() != long.class) throw new IllegalStateException();
+    }
+
+    @Override
+    public boolean writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
+      long l = field.getLong(pojo);
+
+      return vector.getMutator().setSafe(outboundIndex, l);
+    }
+
+  }
+
+  public static class StringWriter extends AbstractWriter<NullableVarCharVector>{
+
+    private ByteBuf data;
+    private final NullableVarCharHolder h = new NullableVarCharHolder();
+
+    public StringWriter(Field field) {
+      super(field, Types.optional(MinorType.VARCHAR));
+      if(field.getType() != String.class) throw new IllegalStateException();
+      ensureLength(100);
+    }
+
+    private void ensureLength(int len){
+      if(data == null || data.capacity() < len){
+        if(data != null) data.release();
+        data = UnpooledByteBufAllocator.DEFAULT.buffer(len);
+      }
+    }
+
+    public void cleanup(){
+      data.release();
+    }
+
+    @Override
+    public boolean writeField(Object pojo, int outboundIndex) throws IllegalArgumentException, IllegalAccessException {
+      String s = (String) field.get(pojo);
+      if(s == null){
+        h.isSet = 0;
+      }else{
+        h.isSet = 1;
+        byte[] bytes = s.getBytes(Charsets.UTF_8);
+        ensureLength(bytes.length);
+        data.clear();
+        data.writeBytes(bytes);
+        h.buffer = data;
+        h.start = 0;
+        h.end = bytes.length;
+
+      }
+
+      return vector.getMutator().setSafe(outboundIndex, h);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
index fb9521f..e5b8ad4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/BatchPrinter.java
@@ -18,7 +18,9 @@
 package org.apache.drill.exec.util;
 
 import com.beust.jcommander.internal.Lists;
+
 import org.apache.commons.lang.StringUtils;
+import org.apache.drill.common.expression.ExpressionStringBuilder;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.record.VectorAccessible;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -37,7 +39,7 @@ public class BatchPrinter {
     List<ValueVector> vectors = Lists.newArrayList();
     int numBatches = 0;
     for (VectorWrapper vw : batch) {
-      columns.add(vw.getValueVectors()[0].getField().getName());
+      columns.add(vw.getValueVectors()[0].getField().getAsSchemaPath().toExpr());
       numBatches = vw.getValueVectors().length;
     }
     int width = columns.size();
@@ -60,7 +62,7 @@ public class BatchPrinter {
     List<String> columns = Lists.newArrayList();
     List<ValueVector> vectors = Lists.newArrayList();
     for (VectorWrapper vw : batch) {
-      columns.add(vw.getValueVector().getField().getName());
+      columns.add(vw.getValueVector().getField().getAsSchemaPath().toExpr());
       vectors.add(vw.getValueVector());
     }
     int width = columns.size();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
index c19faa0..c301ccb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/util/VectorUtil.java
@@ -36,7 +36,7 @@ public class VectorUtil {
     int rows = va.getRecordCount();
     List<String> columns = Lists.newArrayList();
     for (VectorWrapper<?> vw : va) {
-      columns.add(vw.getValueVector().getField().getName());
+      columns.add(vw.getValueVector().getField().getAsSchemaPath().toExpr());
     }
 
     int width = columns.size();
@@ -63,7 +63,7 @@ public class VectorUtil {
         columnCounter++;
       }
     }
-    
+
     for (VectorWrapper<?> vw : va) {
       vw.clear();
     }
@@ -77,7 +77,7 @@ public class VectorUtil {
     int rows = va.getRecordCount();
     List<String> columns = Lists.newArrayList();
     for (VectorWrapper<?> vw : va) {
-      columns.add(vw.getValueVector().getField().getName());
+      columns.add(vw.getValueVector().getField().getAsSchemaPath().toExpr());
     }
 
     int width = columns.size() * (columnWidth + 2);
@@ -113,7 +113,7 @@ public class VectorUtil {
     for (VectorWrapper<?> vw : va) {
       vw.clear();
     }
-    
+
     if (rows > 0 )
       System.out.println(StringUtils.repeat("-", width + 1));
   }


[37/51] [abbrv] git commit: fix NPE in merging receiver

Posted by ja...@apache.org.
fix NPE in merging receiver


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

Branch: refs/heads/master
Commit: 4cfdb3b653ba4db664abc14c4b1d51e4cec5c668
Parents: 0b1df5d
Author: Steven Phillips <sp...@maprtech.com>
Authored: Fri Apr 4 02:51:42 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../impl/mergereceiver/MergingRecordBatch.java       | 15 ++++++++++-----
 1 file changed, 10 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/4cfdb3b6/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index c5c77a6..dcfe02f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -236,22 +236,27 @@ public class MergingRecordBatch implements RecordBatch {
 
       if (node.valueIndex == batchLoaders[node.batchId].getRecordCount() - 1) {
         // reached the end of an incoming record batch
+        RawFragmentBatch nextBatch = null;
         try {
-          incomingBatches[node.batchId] = fragProviders[node.batchId].getNext();
+          nextBatch = fragProviders[node.batchId].getNext();
+
+          while (nextBatch != null && nextBatch.getHeader().getDef().getRecordCount() == 0) {
+            nextBatch = fragProviders[node.batchId].getNext();
+          }
         } catch (IOException e) {
           context.fail(e);
           return IterOutcome.STOP;
         }
 
-        if (incomingBatches[node.batchId].getHeader().getIsLastBatch() ||
-            incomingBatches[node.batchId].getHeader().getDef().getRecordCount() == 0) {
+        incomingBatches[node.batchId] = nextBatch;
+
+        if (nextBatch == null) {
           // batch is empty
-          incomingBatches[node.batchId].release();
           boolean allBatchesEmpty = true;
 
           for (RawFragmentBatch batch : incomingBatches) {
             // see if all batches are empty so we can return OK_* or NONE
-            if (!batch.getHeader().getIsLastBatch()) {
+            if (batch != null) {
               allBatchesEmpty = false;
               break;
             }


[51/51] [abbrv] git commit: fix cleanup methods in ExternalSortBatch and TopNBatch.

Posted by ja...@apache.org.
fix cleanup methods in ExternalSortBatch and TopNBatch.


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

Branch: refs/heads/master
Commit: ba0a6393cea74c89c6f286a88984de6e3ec3762a
Parents: ecd6410
Author: Steven Phillips <sp...@maprtech.com>
Authored: Thu Apr 10 12:11:43 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/physical/impl/TopN/TopNBatch.java  | 8 ++++++--
 .../drill/exec/physical/impl/xsort/ExternalSortBatch.java    | 4 +++-
 2 files changed, 9 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba0a6393/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index 138c1cf..7073a6c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -106,8 +106,12 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
   @Override
   public void cleanup() {
     super.cleanup();
-    sv4.clear();
-    priorityQueue.cleanup();
+    if (sv4 != null) {
+      sv4.clear();
+    }
+    if (priorityQueue != null) {
+      priorityQueue.cleanup();
+    }
     incoming.cleanup();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba0a6393/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 0a3b768..42835fc 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -148,7 +148,9 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
     if (builder != null) {
       builder.clear();
     }
-    sv4.clear();
+    if (sv4 != null) {
+      sv4.clear();
+    }
   }
 
   @Override


[20/51] [abbrv] git commit: handle double project by copying

Posted by ja...@apache.org.
handle double project by copying


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

Branch: refs/heads/master
Commit: 31d199461aaa7bdebe66415bf78b76c98d0fd637
Parents: 2038bc6
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 12:10:12 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../physical/impl/project/ProjectRecordBatch.java | 18 ++++++++++++++----
 1 file changed, 14 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/31d19946/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 949b5f7..4342f52 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -18,7 +18,9 @@
 package org.apache.drill.exec.physical.impl.project;
 
 import java.io.IOException;
+import java.util.HashSet;
 import java.util.List;
+import java.util.Set;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
@@ -118,8 +120,12 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
     final List<TransferPair> transfers = Lists.newArrayList();
     
     final ClassGenerator<Projector> cg = CodeGenerator.getRoot(Projector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
-    
-    if(isAnyWildcard(exprs)){
+
+    Set<Integer> transferFieldIds = new HashSet();
+
+    boolean isAnyWildcard = isAnyWildcard(exprs);
+
+    if(isAnyWildcard){
       for(VectorWrapper<?> wrapper : incoming){
         ValueVector vvIn = wrapper.getValueVector();
         TransferPair tp = wrapper.getValueVector().getTransferPair(new FieldReference(vvIn.getField().getName()));
@@ -134,9 +140,12 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
         if(collector.hasErrors()){
           throw new SchemaChangeException(String.format("Failure while trying to materialize incoming schema.  Errors:\n %s.", collector.toErrorString()));
         }
-        
+
+
         // add value vector to transfer if direct reference and this is allowed, otherwise, add to evaluation stack.
-        if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE){
+        if(expr instanceof ValueVectorReadExpression && incoming.getSchema().getSelectionVectorMode() == SelectionVectorMode.NONE &&
+                !isAnyWildcard &&
+                !transferFieldIds.contains(((ValueVectorReadExpression) expr).getFieldId().getFieldId())) {
           ValueVectorReadExpression vectorRead = (ValueVectorReadExpression) expr;
           ValueVector vvIn = incoming.getValueAccessorById(vectorRead.getFieldId().getFieldId(), TypeHelper.getValueVectorClass(vectorRead.getMajorType().getMinorType(), vectorRead.getMajorType().getMode())).getValueVector();
           Preconditions.checkNotNull(incoming);
@@ -144,6 +153,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
           TransferPair tp = vvIn.getTransferPair(getRef(namedExpression));
           transfers.add(tp);
           container.add(tp.getTo());
+          transferFieldIds.add(vectorRead.getFieldId().getFieldId());
           logger.debug("Added transfer.");
         }else{
           // need to do evaluation.


[27/51] [abbrv] git commit: Adjust project's trait field id based on the mapping from input to output. Output Rel and Logicl Rel for debug purpose.

Posted by ja...@apache.org.
Adjust project's trait field id based on the mapping from input to output.   Output Rel and Logicl Rel for debug purpose.


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

Branch: refs/heads/master
Commit: 342f3fd0d5ded9063f4e8ecacdcf23f7c85b9fa7
Parents: d3c7abe
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Tue Apr 1 11:17:52 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../physical/DrillDistributionTrait.java        |  3 +
 .../exec/planner/physical/ProjectPrule.java     | 86 +++++++++++++++++++-
 .../drill/exec/planner/sql/DrillSqlWorker.java  |  7 +-
 3 files changed, 94 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/342f3fd0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
index aa2dad5..018f548 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
@@ -137,6 +137,9 @@ public class DrillDistributionTrait implements RelTrait {
       return this.fieldId;
     }
     
+    public String toString() {
+      return new Integer(fieldId).toString();
+    }
   }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/342f3fd0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
index 8c6a1e5..fd48058 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
@@ -17,17 +17,36 @@
  */
 package org.apache.drill.exec.planner.physical;
 
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import net.hydromatic.linq4j.Ord;
+
 import org.apache.drill.exec.planner.common.DrillProjectRelBase;
 import org.apache.drill.exec.planner.logical.DrillProjectRel;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionType;
 import org.eigenbase.rel.ProjectRel;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelCollationImpl;
+import org.eigenbase.rel.RelCollationTraitDef;
+import org.eigenbase.rel.RelFieldCollation;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.Convention;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.RelOptRuleCall;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.relopt.volcano.RelSubset;
+import org.eigenbase.rex.RexCall;
+import org.eigenbase.rex.RexInputRef;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.SqlKind;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
 
 public class ProjectPrule extends RelOptRule {
   public static final RelOptRule INSTANCE = new ProjectPrule();
@@ -44,11 +63,21 @@ public class ProjectPrule extends RelOptRule {
     RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL);
     RelNode convertedInput = convert(input, traits);
     
+    Map<Integer, Integer> inToOut = getProjectMap(project);
+    
     if (convertedInput instanceof RelSubset) {
       RelSubset subset = (RelSubset) convertedInput;
       for (RelNode rel : subset.getRelList()) {
         if (!rel.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE).equals(DrillDistributionTrait.DEFAULT)) {
-          call.transformTo(new ProjectPrel(project.getCluster(), rel.getTraitSet(), rel, project.getProjects(), project.getRowType()));
+          DrillDistributionTrait childDist = rel.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE);
+          RelCollation childCollation = rel.getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+          
+          
+          DrillDistributionTrait newDist = convertDist(childDist, inToOut);
+          RelCollation newCollation = convertRelCollation(childCollation, inToOut);
+          
+          call.transformTo(new ProjectPrel(project.getCluster(), project.getTraitSet().plus(newDist).plus(newCollation).plus(Prel.DRILL_PHYSICAL), 
+              rel, project.getProjects(), project.getRowType()));
         }
       }
       
@@ -56,4 +85,59 @@ public class ProjectPrule extends RelOptRule {
       call.transformTo(new ProjectPrel(project.getCluster(), convertedInput.getTraitSet(), convertedInput, project.getProjects(), project.getRowType()));        
     }
   }
+  
+  private DrillDistributionTrait convertDist(DrillDistributionTrait srcDist, Map<Integer, Integer> inToOut) {
+    List<DistributionField> newFields = Lists.newArrayList();
+    
+    for (DistributionField field : srcDist.getFields()) {
+      if (inToOut.containsKey(field.getFieldId())) {
+        newFields.add(new DistributionField(inToOut.get(field.getFieldId())));
+      }
+    }
+    
+    if (newFields.isEmpty()) {
+      if (srcDist.getType() != DistributionType.SINGLETON) {
+        return DrillDistributionTrait.RANDOM_DISTRIBUTED;
+      } else {
+        return DrillDistributionTrait.SINGLETON;
+      }
+    } else {
+      return new DrillDistributionTrait(srcDist.getType(), ImmutableList.copyOf(newFields));
+    }
+  }
+
+  private RelCollation convertRelCollation(RelCollation src, Map<Integer, Integer> inToOut) {
+    List<RelFieldCollation> newFields = Lists.newArrayList();
+    
+    for ( RelFieldCollation field : src.getFieldCollations()) {
+      if (inToOut.containsKey(field.getFieldIndex())) {
+        newFields.add(new RelFieldCollation(inToOut.get(field.getFieldIndex())));
+      }
+    }
+    
+    if (newFields.isEmpty()) {
+      return RelCollationImpl.EMPTY;
+    } else {
+      return RelCollationImpl.of(newFields);
+    }
+  }
+  
+  private Map<Integer, Integer> getProjectMap(DrillProjectRel project) {
+    Map<Integer, Integer> m = new HashMap<Integer, Integer>();
+    
+    for (Ord<RexNode> node : Ord.zip(project.getProjects())) {
+      if (node.e instanceof RexInputRef) {
+        m.put( ((RexInputRef) node.e).getIndex(), node.i);
+      } else if (node.e.isA(SqlKind.CAST)) {
+        RexNode operand = ((RexCall) node.e).getOperands().get(0);
+        if (operand instanceof RexInputRef) {
+          m.put(              
+              ((RexInputRef) operand).getIndex(), node.i);
+        }
+      }
+    }
+    return m;
+    
+  }
+    
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/342f3fd0/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 30eb605..c540066 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -115,13 +115,18 @@ public class DrillSqlWorker {
     
     SqlNode validatedNode = planner.validate(sqlNode);
     RelNode relNode = planner.convert(validatedNode);
+    
+    System.out.println(RelOptUtil.toString(relNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    
     RelNode convertedRelNode = planner.transform(LOGICAL_RULES, planner.getEmptyTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
     if(convertedRelNode instanceof DrillStoreRel){
       throw new UnsupportedOperationException();
     }else{
       convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
     }
-
+    
+    System.out.println(RelOptUtil.toString(convertedRelNode, SqlExplainLevel.ALL_ATTRIBUTES));
+    
     return new RelResult(resultMode, convertedRelNode);
   }
   


[33/51] [abbrv] git commit: fix bug introduced in AggTypes

Posted by ja...@apache.org.
fix bug introduced in AggTypes


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

Branch: refs/heads/master
Commit: 9d233cb7a0a9d3fac52b55fe0b4611ad281d6121
Parents: bfbc83f
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 16:17:21 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/src/main/codegen/data/AggrTypes1.tdd | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/9d233cb7/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index e0a1c11..25dcd96 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -35,7 +35,6 @@
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
       {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
-      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "Float4", runningType: "Float4"},
       {inputType: "Float8", outputType: "Float8", runningType: "Float8"},
@@ -48,7 +47,6 @@
       {inputType: "Int", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
-      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "Float8", runningType: "Float8"},
@@ -62,7 +60,6 @@
       {inputType: "Int", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
-      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "BigInt", runningType: "BigInt"},


[07/51] [abbrv] DRILL-450: Add exchange rules, move from BasicOptimizer to Optiq

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java
deleted file mode 100644
index ffbfa94..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java
+++ /dev/null
@@ -1,47 +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.planner.common;
-
-import java.util.List;
-
-import net.hydromatic.optiq.prepare.Prepare.CatalogReader;
-
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableModificationRelBase;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelTraitSet;
-
-public class DrillStoreRel extends TableModificationRelBase implements DrillRel{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRel.class);
-
-  protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,
-      RelNode child, Operation operation, List<String> updateColumnList, boolean flattened) {
-    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
-    
-  }
-
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.java
new file mode 100644
index 0000000..c6e3ae1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRelBase.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.planner.common;
+
+import java.util.List;
+
+import net.hydromatic.optiq.prepare.Prepare.CatalogReader;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.TableModificationRelBase;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+
+/**
+ * Base class for logical and physical Store implemented in Drill
+ */
+public abstract class DrillStoreRelBase extends TableModificationRelBase implements DrillRelNode {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRelBase.class);
+
+  protected DrillStoreRelBase(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,
+      RelNode child, Operation operation, List<String> updateColumnList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
+    
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
deleted file mode 100644
index 2794560..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
+++ /dev/null
@@ -1,73 +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.planner.common;
-
-import java.util.List;
-
-import net.hydromatic.linq4j.Ord;
-
-import org.apache.drill.common.logical.data.Limit;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.common.logical.data.Union;
-import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillRel;
-import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.UnionRelBase;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
-
-/**
- * Union implemented in Drill.
- */
-public class DrillUnionRel extends UnionRelBase implements DrillRel {
-  /** Creates a DrillUnionRel. */
-  public DrillUnionRel(RelOptCluster cluster, RelTraitSet traits,
-      List<RelNode> inputs, boolean all) {
-    super(cluster, traits, inputs, all);
-  }
-
-  @Override
-  public DrillUnionRel copy(RelTraitSet traitSet, List<RelNode> inputs,
-      boolean all) {
-    return new DrillUnionRel(getCluster(), traitSet, inputs, all);
-  }
-
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    // divide cost by two to ensure cheaper than EnumerableDrillRel
-    return super.computeSelfCost(planner).multiplyBy(.5);
-  }
-
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    Union.Builder builder = Union.builder();
-    for (Ord<RelNode> input : Ord.zip(inputs)) {
-      builder.addInput(implementor.visitChild(this, input.i, input.e));
-    }
-    builder.setDistinct(!all);
-    return builder.build();
-  }
-  
-  public static DrillUnionRel convert(Union union, ConversionContext context) throws InvalidRelException{
-    throw new UnsupportedOperationException();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRelBase.java
new file mode 100644
index 0000000..a16b8ee
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRelBase.java
@@ -0,0 +1,37 @@
+/**
+ * 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.common;
+
+import java.util.List;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.UnionRelBase;
+import org.eigenbase.relopt.RelOptCluster;
+
+import org.eigenbase.relopt.RelTraitSet;
+
+/**
+ * Base class for logical and physical Union implemented in Drill
+ */
+public abstract class DrillUnionRelBase extends UnionRelBase implements DrillRelNode {
+ 
+  public DrillUnionRelBase(RelOptCluster cluster, RelTraitSet traits,
+      List<RelNode> inputs, boolean all) {
+    super(cluster, traits, inputs, all);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
index e1337ef..c2833c1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRel.java
@@ -30,6 +30,7 @@ import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.common.logical.data.GroupingAggregate;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.common.DrillAggregateRelBase;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.AggregateCall;
 import org.eigenbase.rel.AggregateRelBase;
@@ -43,7 +44,7 @@ import com.google.common.collect.Lists;
 /**
  * Aggregation implemented in Drill.
  */
-public class DrillAggregateRel extends AggregateRelBase implements DrillRel {
+public class DrillAggregateRel extends DrillAggregateRelBase implements DrillRel {
   /** Creates a DrillAggregateRel. */
   public DrillAggregateRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
       List<AggregateCall> aggCalls) throws InvalidRelException {
@@ -55,10 +56,9 @@ public class DrillAggregateRel extends AggregateRelBase implements DrillRel {
     }
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+  public AggregateRelBase copy(RelTraitSet traitSet, RelNode input, BitSet groupSet, List<AggregateCall> aggCalls) {
     try {
-      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+      return new DrillAggregateRel(getCluster(), traitSet, input, getGroupSet(), aggCalls);
     } catch (InvalidRelException e) {
       throw new AssertionError(e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
index 867def4..fee62c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
@@ -21,7 +21,7 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.common.BaseFilterRel;
+import org.apache.drill.exec.planner.common.DrillFilterRelBase;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;
@@ -30,7 +30,7 @@ import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexNode;
 
 
-public class DrillFilterRel extends BaseFilterRel implements DrillRel {
+public class DrillFilterRel extends DrillFilterRelBase implements DrillRel {
   protected DrillFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
     super(DRILL_LOGICAL, cluster, traits, child, condition);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/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 16e9d47..31a8e6e 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
@@ -27,9 +27,9 @@ 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.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.JoinRelBase;
 import org.eigenbase.rel.JoinRelType;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
@@ -45,14 +45,11 @@ import org.eigenbase.util.Pair;
 /**
  * Join implemented in Drill.
  */
-public class DrillJoinRel extends JoinRelBase implements DrillRel {
-  private final List<Integer> leftKeys = new ArrayList<>();
-  private final List<Integer> rightKeys = new ArrayList<>();
-
+public class DrillJoinRel extends DrillJoinRelBase implements DrillRel {
   /** Creates a DrillJoinRel. */
   public DrillJoinRel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType) throws InvalidRelException {
-    super(cluster, traits, left, right, condition, joinType, Collections.<String> emptySet());
+    super(cluster, traits, left, right, condition, joinType);
 
     RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
     if (!remaining.isAlwaysTrue()) {
@@ -88,7 +85,8 @@ public class DrillJoinRel extends JoinRelBase implements DrillRel {
     for (Pair<Integer, Integer> pair : Pair.zip(leftKeys, rightKeys)) {
       builder.addCondition("==", new FieldReference(leftFields.get(pair.left)), new FieldReference(rightFields.get(pair.right)));
     }
-    return builder.build();
+    
+    return builder.build();  
   }
 
   /**
@@ -146,20 +144,4 @@ public class DrillJoinRel extends JoinRelBase implements DrillRel {
     return new DrillJoinRel(context.getCluster(), context.getLogicalTraits(), left, right, rexCondition, join.getJoinType());
   }
   
-  
-  /**
-   * Returns whether there are any elements in common between left and right.
-   */
-  private static <T> boolean intersects(List<T> left, List<T> right) {
-    return new HashSet<>(left).removeAll(right);
-  }
-
-  private boolean uniqueFieldNames(RelDataType rowType) {
-    return isUnique(rowType.getFieldNames());
-  }
-
-  private static <T> boolean isUnique(List<T> list) {
-    return new HashSet<>(list).size() == list.size();
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
index 137df72..9f08117 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
@@ -22,26 +22,22 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Limit;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.common.DrillLimitRelBase;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexLiteral;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.sql.type.SqlTypeName;
 
-public class DrillLimitRel extends SingleRel implements DrillRel {
-  private RexNode offset;
-  private RexNode fetch;
+public class DrillLimitRel extends DrillLimitRelBase implements DrillRel {
 
   public DrillLimitRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
-    super(cluster, traitSet, child);
-    this.offset = offset;
-    this.fetch = fetch;
+    super(cluster, traitSet, child, offset, fetch);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
index f1d3bae..d8396b5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
@@ -17,7 +17,6 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.drill.exec.planner.common.DrillLimitRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.SortRel;
 import org.eigenbase.relopt.Convention;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
index eb9086d..ae777cb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
@@ -23,9 +23,10 @@ import java.util.List;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.logical.data.Project;
-import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.ProjectRelBase;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
@@ -41,15 +42,15 @@ import com.google.common.collect.Lists;
 /**
  * Project implemented in Drill.
  */
-public class DrillProjectRel extends BaseProjectRel implements DrillRel {
+public class DrillProjectRel extends DrillProjectRelBase implements DrillRel {
   protected DrillProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
       RelDataType rowType) {
     super(DRILL_LOGICAL, cluster, traits, child, exps, rowType);
   }
 
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new DrillProjectRel(getCluster(), traitSet, sole(inputs), new ArrayList<RexNode>(exps), rowType);
+
+  public ProjectRelBase copy(RelTraitSet traitSet, RelNode input, List<RexNode> exps, RelDataType rowType) {
+    return new DrillProjectRel(getCluster(), traitSet, input, exps, rowType);
   }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
index 153b147..7eca54e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
@@ -18,16 +18,16 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.drill.common.logical.data.LogicalOperator;
-import org.eigenbase.rel.RelNode;
+import org.apache.drill.exec.planner.common.DrillRelNode;
 import org.eigenbase.relopt.Convention;
 
 /**
  * Relational expression that is implemented in Drill.
  */
-public interface DrillRel extends RelNode {
+public interface DrillRel extends DrillRelNode {
   /** Calling convention for relational expressions that are "implemented" by
    * generating Drill logical plans. */
-  Convention DRILL_LOGICAL = new Convention.Impl("DRILL", DrillRel.class);
+  Convention DRILL_LOGICAL = new Convention.Impl("LOGICAL", DrillRel.class);
 
   LogicalOperator implement(DrillImplementor implementor);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/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 5bd1386..9ab5308 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,23 +21,17 @@ import java.util.Iterator;
 
 import net.hydromatic.optiq.tools.RuleSet;
 
+import org.apache.drill.exec.planner.physical.FilterPrule;
+import org.apache.drill.exec.planner.physical.LimitPrule;
+import org.apache.drill.exec.planner.physical.MergeJoinPrule;
 import org.apache.drill.exec.planner.physical.ProjectPrule;
 import org.apache.drill.exec.planner.physical.ScanPrule;
-import org.eigenbase.rel.rules.MergeProjectRule;
-import org.eigenbase.rel.rules.PushFilterPastJoinRule;
-import org.eigenbase.rel.rules.PushFilterPastProjectRule;
-import org.eigenbase.rel.rules.PushJoinThroughJoinRule;
-import org.eigenbase.rel.rules.PushSortPastProjectRule;
-import org.eigenbase.rel.rules.ReduceAggregatesRule;
-import org.eigenbase.rel.rules.RemoveDistinctAggregateRule;
-import org.eigenbase.rel.rules.RemoveDistinctRule;
-import org.eigenbase.rel.rules.RemoveSortRule;
-import org.eigenbase.rel.rules.RemoveTrivialCalcRule;
-import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
-import org.eigenbase.rel.rules.SwapJoinRule;
-import org.eigenbase.rel.rules.TableAccessRule;
-import org.eigenbase.rel.rules.UnionToDistinctRule;
+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.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.volcano.AbstractConverter;
 import org.eigenbase.relopt.volcano.AbstractConverter.ExpandConversionRule;
 
 import com.google.common.collect.ImmutableSet;
@@ -59,17 +53,26 @@ public class DrillRuleSets {
       ));
   
   public static final RuleSet DRILL_PHYSICAL_MEM = new DrillRuleSet(ImmutableSet.of( //
-      DrillScanRule.INSTANCE,
-      DrillFilterRule.INSTANCE,
-      DrillProjectRule.INSTANCE,
-      DrillAggregateRule.INSTANCE,
+//      DrillScanRule.INSTANCE,
+//      DrillFilterRule.INSTANCE,
+//      DrillProjectRule.INSTANCE,
+//      DrillAggregateRule.INSTANCE,
+//
+//      DrillLimitRule.INSTANCE,
+//      DrillSortRule.INSTANCE,
+//      DrillJoinRule.INSTANCE,
+//      DrillUnionRule.INSTANCE,
 
-      DrillLimitRule.INSTANCE,
-      DrillSortRule.INSTANCE,
-      DrillJoinRule.INSTANCE,
-      DrillUnionRule.INSTANCE,
+      SortConvertPrule.INSTANCE,
+      SortPrule.INSTANCE,
       ProjectPrule.INSTANCE,
-      ScanPrule.INSTANCE
+      ScanPrule.INSTANCE,
+      ScreenPrule.INSTANCE,
+      ExpandConversionRule.INSTANCE,
+      StreamAggPrule.INSTANCE,
+      MergeJoinPrule.INSTANCE,
+      FilterPrule.INSTANCE,
+      LimitPrule.INSTANCE
 
 //    ExpandConversionRule.instance,
 //    SwapJoinRule.instance,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
index 2629c77..619e76d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
@@ -20,7 +20,7 @@ package org.apache.drill.exec.planner.logical;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Scan;
-import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptTable;
@@ -29,14 +29,14 @@ import org.eigenbase.relopt.RelTraitSet;
 /**
  * GroupScan of a Drill table.
  */
-public class DrillScanRel extends BaseScanRel implements DrillRel {
-  private final DrillTable drillTable;
+public class DrillScanRel extends DrillScanRelBase implements DrillRel {
+  //private final DrillTable drillTable;
 
   /** Creates a DrillScan. */
   public DrillScanRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
     super(DRILL_LOGICAL, cluster, traits, table);
-    this.drillTable = table.unwrap(DrillTable.class);
-    assert drillTable != null;
+    //this.drillTable = table.unwrap(DrillTable.class);
+    //assert drillTable != null;
   }
 
   public LogicalOperator implement(DrillImplementor implementor) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
index e472daa..4896ac9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
@@ -19,7 +19,7 @@ package org.apache.drill.exec.planner.logical;
 
 import net.hydromatic.optiq.rules.java.JavaRules.EnumerableTableAccessRel;
 
-import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.planner.common.DrillScanRelBase;
 import org.eigenbase.relopt.Convention;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.RelOptRuleCall;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
index abfcd10..a63cae0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
@@ -21,14 +21,14 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Store;
-import org.apache.drill.exec.planner.common.BaseScreenRel;
+import org.apache.drill.exec.planner.common.DrillScreenRelBase;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class DrillScreenRel extends BaseScreenRel implements DrillRel {
+public class DrillScreenRel extends DrillScreenRelBase implements DrillRel {
   private static final Logger logger = LoggerFactory.getLogger(DrillScreenRel.class);
 
   public DrillScreenRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
index 8f9841c..830c34b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
@@ -22,15 +22,15 @@ import java.util.List;
 import net.hydromatic.optiq.prepare.Prepare.CatalogReader;
 
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.common.DrillStoreRelBase;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.TableModificationRelBase;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptTable;
 import org.eigenbase.relopt.RelTraitSet;
 
-public class DrillStoreRel extends TableModificationRelBase implements DrillRel{
+public class DrillStoreRel extends DrillStoreRelBase implements DrillRel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRel.class);
 
   protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index c111306..8e6b914 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -27,7 +27,6 @@ import net.hydromatic.optiq.Table;
 import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.physical.base.GroupScan;
-import org.apache.drill.exec.planner.common.BaseScanRel;
 import org.apache.drill.exec.store.StoragePlugin;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptTable;
@@ -39,15 +38,24 @@ public abstract class DrillTable implements Table{
   public final StoragePluginConfig storageEngineConfig;
   private Object selection;
   private StoragePlugin plugin;
+  private GroupScan scan; 
   
   /** Creates a DrillTable. */
   public DrillTable(String storageEngineName, StoragePlugin plugin, Object selection) {
     this.selection = selection;
+    this.plugin = plugin;
     
     this.storageEngineConfig = plugin.getConfig();
     this.storageEngineName = storageEngineName;
   }
 
+  public GroupScan getGroupScan() throws IOException{
+    if(scan == null){
+      this.scan = plugin.getPhysicalScan(new JSONOptions(selection));
+    }
+    return scan;
+  }
+  
   public StoragePluginConfig getStorageEngineConfig(){
     return storageEngineConfig;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
index c366341..93c4ca7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
@@ -21,15 +21,13 @@ import java.util.List;
 
 import net.hydromatic.linq4j.Ord;
 
-import org.apache.drill.common.logical.data.Limit;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Union;
+import org.apache.drill.exec.planner.common.DrillUnionRelBase;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.UnionRelBase;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptCost;
 import org.eigenbase.relopt.RelOptPlanner;
@@ -38,7 +36,7 @@ import org.eigenbase.relopt.RelTraitSet;
 /**
  * Union implemented in Drill.
  */
-public class DrillUnionRel extends UnionRelBase implements DrillRel {
+public class DrillUnionRel extends DrillUnionRelBase implements DrillRel {
   /** Creates a DrillUnionRel. */
   public DrillUnionRel(RelOptCluster cluster, RelTraitSet traits,
       List<RelNode> inputs, boolean all) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
index 99f2c98..25515c0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
-import org.apache.drill.exec.planner.common.DrillUnionRel;
+import org.apache.drill.exec.planner.common.DrillUnionRelBase;
 import org.eigenbase.rel.UnionRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.*;
@@ -26,7 +26,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 /**
- * Rule that converts a {@link UnionRel} to a {@link DrillUnionRel}, implemented by a "union" operation.
+ * Rule that converts a {@link UnionRel} to a {@link DrillUnionRelBase}, implemented by a "union" operation.
  */
 public class DrillUnionRule extends RelOptRule {
   public static final RelOptRule INSTANCE = new DrillUnionRule();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
index d309405..b830b1a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
@@ -18,7 +18,9 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.drill.common.logical.StoragePluginConfig;
-import org.apache.drill.exec.store.dfs.FileSystemPlugin;
+import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
+import org.apache.drill.exec.planner.types.RelDataTypeHolder;
+import org.apache.drill.exec.store.StoragePlugin;
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.reltype.RelDataTypeFactory;
 
@@ -27,7 +29,7 @@ public class DynamicDrillTable extends DrillTable{
 
   private RelDataTypeHolder holder = new RelDataTypeHolder();
   
-  public DynamicDrillTable(FileSystemPlugin plugin, String storageEngineName, Object selection, StoragePluginConfig storageEngineConfig) {
+  public DynamicDrillTable(StoragePlugin plugin, String storageEngineName, Object selection, StoragePluginConfig storageEngineConfig) {
     super(storageEngineName, plugin, selection);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
index d54ee79..efc29f4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
@@ -19,12 +19,12 @@ package org.apache.drill.exec.planner.logical;
 
 import net.hydromatic.optiq.rules.java.EnumerableConvention;
 
-import org.apache.drill.exec.planner.common.BaseScreenRel;
+import org.apache.drill.exec.planner.common.DrillScreenRelBase;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.convert.ConverterRule;
 
 /**
- * Rule that converts any Drill relational expression to enumerable format by adding a {@link BaseScreenRel}.
+ * Rule that converts any Drill relational expression to enumerable format by adding a {@link DrillScreenRelBase}.
  */
 public class EnumerableDrillRule extends ConverterRule {
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeDrillImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeDrillImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeDrillImpl.java
deleted file mode 100644
index bc0f34e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeDrillImpl.java
+++ /dev/null
@@ -1,78 +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.planner.logical;
-
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelDataTypeImpl;
-import org.eigenbase.sql.type.SqlTypeName;
-
-/* We use an instance of this class as the row type for
- * Drill table. Since we don't know the schema before hand
- * whenever optiq requires us to validate that a field exists
- * we always return true and indicate that the type of that
- * field is 'ANY'
- */
-public class RelDataTypeDrillImpl extends RelDataTypeImpl {
-
-    private final RelDataTypeFactory typeFactory;
-    private final RelDataTypeHolder holder;
-    
-    public RelDataTypeDrillImpl(RelDataTypeHolder holder, RelDataTypeFactory typeFactory) {
-        this.typeFactory = typeFactory;
-        this.holder = holder;
-        computeDigest();
-    }
-    
-    @Override
-    public List<RelDataTypeField> getFieldList() {
-      return holder.getFieldList(typeFactory);
-    }
-    
-    @Override
-    public int getFieldCount() {
-      return holder.getFieldCount();
-    }
-
-    @Override
-    public RelDataTypeField getField(String fieldName, boolean caseSensitive) {
-      return holder.getField(typeFactory, fieldName);
-    }
-
-    @Override
-    public List<String> getFieldNames() {
-      return holder.getFieldNames();
-    }
-
-    @Override
-    public SqlTypeName getSqlTypeName() {
-        return null;
-    }
-
-    @Override
-    protected void generateTypeString(StringBuilder sb, boolean withDetail) {
-       sb.append("DrillRecordRow");
-    }
-
-    @Override
-    public boolean isStruct() {
-        return true;
-    }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeHolder.java
deleted file mode 100644
index 038f3de..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/RelDataTypeHolder.java
+++ /dev/null
@@ -1,85 +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.planner.logical;
-
-import java.util.List;
-
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelDataTypeField;
-import org.eigenbase.reltype.RelDataTypeFieldImpl;
-import org.eigenbase.sql.type.SqlTypeName;
-
-import com.google.common.collect.Lists;
-
-public class RelDataTypeHolder {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(RelDataTypeHolder.class);
-
-  List<String> fieldNames = Lists.newArrayList();
-
-  public List<RelDataTypeField> getFieldList(RelDataTypeFactory typeFactory) {
-
-    addStarIfEmpty();
-
-    List<RelDataTypeField> fields = Lists.newArrayList();
-
-    int i = 0;
-    for (String fieldName : fieldNames) {
-
-      RelDataTypeField field = new RelDataTypeFieldImpl(fieldName, i, typeFactory.createSqlType(SqlTypeName.ANY));
-      fields.add(field);
-      i++;
-    }
-
-    return fields;
-  }
-
-  public int getFieldCount() {
-    addStarIfEmpty();
-    return fieldNames.size();
-  }
-
-  private void addStarIfEmpty(){
-    if (fieldNames.isEmpty()) fieldNames.add("*");
-  }
-  
-  public RelDataTypeField getField(RelDataTypeFactory typeFactory, String fieldName) {
-
-    /* First check if this field name exists in our field list */
-    int i = 0;
-    for (String name : fieldNames) {
-      if (name.equalsIgnoreCase(fieldName)) {
-        return new RelDataTypeFieldImpl(name, i, typeFactory.createSqlType(SqlTypeName.ANY));
-      }
-      i++;
-    }
-
-    /* This field does not exist in our field list add it */
-    RelDataTypeField newField = new RelDataTypeFieldImpl(fieldName, fieldNames.size(),
-        typeFactory.createSqlType(SqlTypeName.ANY));
-
-    /* Add the name to our list of field names */
-    fieldNames.add(fieldName);
-
-    return newField;
-  }
-
-  public List<String> getFieldNames() {
-    addStarIfEmpty();
-    return fieldNames;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
new file mode 100644
index 0000000..aa2dad5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
@@ -0,0 +1,142 @@
+/**
+ * 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.physical;
+
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.relopt.RelTrait;
+import org.eigenbase.relopt.RelTraitDef;
+
+import com.google.common.collect.ImmutableList;
+
+public class DrillDistributionTrait implements RelTrait {
+  public static enum DistributionType {SINGLETON, HASH_DISTRIBUTED, RANGE_DISTRIBUTED, RANDOM_DISTRIBUTED,
+                                       ROUND_ROBIN_DISTRIBUTED, BROADCAST_DISTRIBUTED, ANY};
+
+  public static DrillDistributionTrait SINGLETON = new DrillDistributionTrait(DistributionType.SINGLETON);
+  public static DrillDistributionTrait RANDOM_DISTRIBUTED = new DrillDistributionTrait(DistributionType.RANDOM_DISTRIBUTED);
+  public static DrillDistributionTrait ANY = new DrillDistributionTrait(DistributionType.ANY);
+  
+  public static DrillDistributionTrait DEFAULT = ANY;
+  
+  private DistributionType type;  
+  private final ImmutableList<DistributionField> fields;
+  
+  private DrillDistributionTrait(DistributionType type) {
+    assert (type == DistributionType.SINGLETON || type == DistributionType.RANDOM_DISTRIBUTED || type == DistributionType.ANY
+            || type == DistributionType.ROUND_ROBIN_DISTRIBUTED || type == DistributionType.BROADCAST_DISTRIBUTED);
+    this.type = type;
+    this.fields = ImmutableList.<DistributionField>of();
+  }
+
+  public DrillDistributionTrait(DistributionType type, ImmutableList<DistributionField> fields) {
+    assert (type == DistributionType.HASH_DISTRIBUTED || type == DistributionType.RANGE_DISTRIBUTED);   
+    this.type = type;
+    this.fields = fields;
+  }
+
+  public boolean subsumes(RelTrait trait) {
+
+    if (trait instanceof DrillDistributionTrait) {
+      DistributionType requiredDist = ((DrillDistributionTrait) trait).getType();
+      if (requiredDist == DistributionType.ANY) {
+        return true;
+      }
+
+      if (this.type == DistributionType.HASH_DISTRIBUTED) {
+        if (requiredDist == DistributionType.HASH_DISTRIBUTED) {
+          ImmutableList<DistributionField> thisFields = this.fields;
+          ImmutableList<DistributionField> requiredFields = ((DrillDistributionTrait)trait).getFields();
+
+          assert(thisFields.size() > 0 && requiredFields.size() > 0);
+
+          // A subset of the required distribution columns can satisfy (subsume) the requirement
+          // e.g: required distribution: {a, b, c} 
+          // Following can satisfy the requirements: {a}, {b}, {c}, {a, b}, {b, c}, {a, c} or {a, b, c}
+          return (requiredFields.containsAll(thisFields));
+        }
+        else if (requiredDist == DistributionType.RANDOM_DISTRIBUTED) {
+          return true; // hash distribution subsumes random distribution and ANY distribution 
+        }
+      }
+    }
+
+    return this.equals(trait);
+  }
+  
+  public RelTraitDef<DrillDistributionTrait> getTraitDef() {
+    return DrillDistributionTraitDef.INSTANCE;
+  }
+
+  public DistributionType getType() {
+    return this.type;
+  }
+
+  public ImmutableList<DistributionField> getFields() {
+    return fields;
+  }
+
+  public int hashCode() {
+    return  fields == null ? type.hashCode() : type.hashCode() | fields.hashCode() << 4 ;
+  }
+  
+  public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj instanceof DrillDistributionTrait) {
+      DrillDistributionTrait that = (DrillDistributionTrait) obj;
+      return this.type == that.type && this.fields.equals(that.fields) ;
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return fields == null ? this.type.toString() : this.type.toString() + "(" + fields + ")";
+  }
+
+  
+  public static class DistributionField {
+    /**
+     * 0-based index of field being DISTRIBUTED.
+     */
+    private final int fieldId;
+    
+    public DistributionField (int fieldId) {
+      this.fieldId = fieldId;
+    }
+
+    public boolean equals(Object obj) {
+      if (!(obj instanceof DistributionField)) {
+        return false;
+      }
+      DistributionField other = (DistributionField) obj;
+      return this.fieldId == other.fieldId;
+    }
+    
+    public int hashCode() {
+      return this.fieldId;
+    }
+    
+    public int getFieldId() {
+      return this.fieldId;
+    }
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/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
new file mode 100644
index 0000000..c2ebb7a
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTraitDef.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.planner.physical;
+
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelCollationImpl;
+import org.eigenbase.rel.RelCollationTraitDef;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitDef;
+
+public class DrillDistributionTraitDef extends RelTraitDef<DrillDistributionTrait>{
+  public static final DrillDistributionTraitDef INSTANCE = new DrillDistributionTraitDef();
+  
+  private DrillDistributionTraitDef() {
+    super();
+  }
+  
+  public boolean canConvert(
+      RelOptPlanner planner, DrillDistributionTrait fromTrait, DrillDistributionTrait toTrait) {
+    return true;
+  }  
+
+  public Class<DrillDistributionTrait> getTraitClass(){
+    return DrillDistributionTrait.class;
+  }
+  
+  public DrillDistributionTrait getDefault() {
+    return DrillDistributionTrait.DEFAULT;
+  }
+
+  public String getSimpleName() {
+    return this.getClass().getSimpleName();
+  }
+
+  // implement RelTraitDef
+  public RelNode convert(
+      RelOptPlanner planner,
+      RelNode rel,
+      DrillDistributionTrait toDist,
+      boolean allowInfiniteCostConverters) {
+    
+    DrillDistributionTrait currentDist = rel.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE);
+    
+    // Source and Target have the same trait.
+    if (currentDist.equals(toDist)) {
+      return rel;
+    }
+    
+    // Source trait is "ANY", which is abstract type of distribution.
+    // We do not want to convert from "ANY", since it's abstract. 
+    // Source trait should be concrete type: SINGLETON, HASH_DISTRIBUTED, etc.
+    if (currentDist.equals(DrillDistributionTrait.DEFAULT)) {
+      return null;
+    }
+    
+    RelCollation collation = null;
+    
+    switch(toDist.getType()){
+      // UnionExchange, HashToRandomExchange, OrderedPartitionExchange destroy the ordering property, therefore RelCollation is set to default, which is EMPTY.
+      case SINGLETON:         
+        return new UnionExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
+      case HASH_DISTRIBUTED:
+        return new HashToRandomExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel, toDist.getFields());
+      case RANGE_DISTRIBUTED:
+        return new OrderedPartitionExchangePrel(rel.getCluster(), planner.emptyTraitSet().plus(Prel.DRILL_PHYSICAL).plus(toDist), rel);
+      default:
+        return null;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
new file mode 100644
index 0000000..f392a18
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
@@ -0,0 +1,58 @@
+/**
+ * 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.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.planner.common.DrillFilterRelBase;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
+
+
+public class FilterPrel extends DrillFilterRelBase implements Prel {
+  protected FilterPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+    super(Prel.DRILL_PHYSICAL, cluster, traits, child, condition);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new FilterPrel(getCluster(), traitSet, sole(inputs), getCondition());
+  }
+  
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, Filter only accepts "NONE", SV2, SV4. 
+    
+    Filter p = new Filter(childPOP, getFilterExpression(new DrillParseContext()), 1.0f);
+    creator.addPhysicalOperator(p);
+    
+    return p;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrule.java
new file mode 100644
index 0000000..9496ba2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrule.java
@@ -0,0 +1,58 @@
+/**
+ * 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.physical;
+
+import org.apache.drill.exec.planner.common.DrillProjectRelBase;
+import org.apache.drill.exec.planner.logical.DrillFilterRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.rel.ProjectRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.relopt.volcano.RelSubset;
+
+public class FilterPrule extends RelOptRule {
+  public static final RelOptRule INSTANCE = new FilterPrule();
+
+  private FilterPrule() {
+    super(RelOptHelper.some(DrillFilterRel.class, RelOptHelper.any(RelNode.class)), "FilterPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillFilterRel  filter = (DrillFilterRel) call.rel(0);
+    final RelNode input = call.rel(1);
+
+    RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL);
+    RelNode convertedInput = convert(input, traits);
+    
+    if (convertedInput instanceof RelSubset) {
+      RelSubset subset = (RelSubset) convertedInput;
+      for (RelNode rel : subset.getRelList()) {
+        if (!rel.getTraitSet().getTrait(DrillDistributionTraitDef.INSTANCE).equals(DrillDistributionTrait.DEFAULT)) {
+          call.transformTo(new FilterPrel(filter.getCluster(), rel.getTraitSet(), convertedInput, filter.getCondition()));
+        }
+      }      
+    } else{
+      call.transformTo(new FilterPrel(filter.getCluster(), convertedInput.getTraitSet(), convertedInput, filter.getCondition()));        
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
deleted file mode 100644
index 27b6be9..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
+++ /dev/null
@@ -1,37 +0,0 @@
-package org.apache.drill.exec.planner.physical;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.apache.drill.exec.planner.logical.DrillAggregateRel;
-import org.eigenbase.rel.AggregateCall;
-import org.eigenbase.rel.AggregateRelBase;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-
-public class HashPrel extends AggregateRelBase implements Prel{
-
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPrel.class);
-
-  public HashPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
-      List<AggregateCall> aggCalls) throws InvalidRelException {
-    super(cluster, traits, child, groupSet, aggCalls);
-    for (AggregateCall aggCall : aggCalls) {
-      if (aggCall.isDistinct()) {
-        throw new InvalidRelException("HashPrel does not support DISTINCT aggregates");
-      }
-    }
-    assert getConvention() == DRILL_PHYSICAL;
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    try {
-      return new HashPrel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
-    } catch (InvalidRelException e) {
-      throw new AssertionError(e);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
new file mode 100644
index 0000000..c2b6c68
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -0,0 +1,77 @@
+/**
+ * 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.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.HashToRandomExchange;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+
+public class HashToRandomExchangePrel extends SingleRel implements Prel {
+
+  private final List<DistributionField> fields;
+  
+  public HashToRandomExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, List<DistributionField> fields) {
+    super(cluster, traitSet, input);
+    this.fields = fields;
+    assert input.getConvention() == Prel.DRILL_PHYSICAL;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(.1);    
+    //return planner.getCostFactory().makeZeroCost();
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new HashToRandomExchangePrel(getCluster(), traitSet, sole(inputs), fields);
+  }
+  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
+      childPOP = new SelectionVectorRemover(childPOP);
+      creator.addPhysicalOperator(childPOP);
+    }
+
+    HashToRandomExchange g = new HashToRandomExchange(childPOP, PrelUtil.getHashExpression(this.fields, getChild().getRowType()));
+    creator.addPhysicalOperator(g);
+    return g;    
+  }
+  
+  public List<DistributionField> getFields() {
+    return this.fields;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
new file mode 100644
index 0000000..f2f46c1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
@@ -0,0 +1,95 @@
+/**
+ * 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.physical;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Filter;
+import org.apache.drill.exec.physical.config.Limit;
+import org.apache.drill.exec.planner.common.DrillLimitRelBase;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.type.SqlTypeName;
+
+public class LimitPrel extends DrillLimitRelBase implements Prel {
+
+  public LimitPrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
+    super(cluster, traitSet, child, offset, fetch);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new LimitPrel(getCluster(), traitSet, sole(inputs), offset, fetch);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+    
+    // First offset to include into results (inclusive). Null implies it is starting from offset 0
+    int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
+
+    // Last offset to stop including into results (exclusive), translating fetch row counts into an offset.
+    // Null value implies including entire remaining result set from first offset
+    Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
+
+    Limit limit = new Limit(childPOP, first, last);
+    
+    creator.addPhysicalOperator(limit);
+    
+    return limit;
+  }
+  
+//  @Override
+//  public LogicalOperator implement(DrillImplementor implementor) {
+//    LogicalOperator inputOp = implementor.visitChild(this, 0, getChild());
+//    
+//    // First offset to include into results (inclusive). Null implies it is starting from offset 0
+//    int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
+//
+//    // Last offset to stop including into results (exclusive), translating fetch row counts into an offset.
+//    // Null value implies including entire remaining result set from first offset
+//    Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
+//    Limit limit = new Limit(first, last);
+//    limit.setInput(inputOp);
+//    return limit;
+//  }
+  
+//  public static LimitPrel convert(Limit limit, ConversionContext context) throws InvalidRelException{
+//    RelNode input = context.toRel(limit.getInput());
+//    RexNode first = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getFirst()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+//    RexNode last = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getLast()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+//    return new LimitPrel(context.getCluster(), context.getLogicalTraits(), input, first, last);
+//  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrule.java
new file mode 100644
index 0000000..8076db5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrule.java
@@ -0,0 +1,50 @@
+/**
+ * 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.physical;
+
+import org.apache.drill.exec.planner.common.DrillScreenRelBase;
+import org.apache.drill.exec.planner.logical.DrillLimitRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class LimitPrule extends RelOptRule{
+  public static final RelOptRule INSTANCE = new LimitPrule();
+
+  
+  public LimitPrule() {
+    super(RelOptHelper.some(DrillLimitRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "Prel.LimitPrule");    
+  }
+  
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillLimitRel limit = (DrillLimitRel) call.rel(0);
+    final RelNode input = call.rel(1);
+    
+    final RelTraitSet traits = input.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    final RelNode convertedInput = convert(input, traits);
+    LimitPrel newLimit = new LimitPrel(limit.getCluster(), limit.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON), convertedInput, limit.getOffset(), limit.getFetch());
+    call.transformTo(newLimit);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
new file mode 100644
index 0000000..e21ab4d
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.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.planner.physical;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.logical.data.JoinCondition;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.MergeJoinPOP;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
+import org.apache.drill.exec.planner.common.DrillJoinRelBase;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinRelBase;
+import org.eigenbase.rel.JoinRelType;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptUtil;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.util.Pair;
+
+import com.beust.jcommander.internal.Lists;
+
+public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
+
+  //private final JoinCondition[] joinConditions; // Drill's representation of join conditions
+  
+  /** Creates a MergeJoiPrel. */
+  public MergeJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
+      JoinRelType joinType) throws InvalidRelException {
+    super(cluster, traits, left, right, condition, joinType);
+
+    RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
+    if (!remaining.isAlwaysTrue()) {
+      throw new InvalidRelException("MergeJoinPrel only supports equi-join");
+    }
+    //this.joinConditions = joinConditions;
+  }
+
+  
+  @Override 
+  public JoinRelBase copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, JoinRelType joinType) {
+    try {
+      return new MergeJoinPrel(this.getCluster(), traitSet, left, right, conditionExpr, joinType);
+    }catch (InvalidRelException e) {
+      throw new AssertionError(e);
+    }
+  }
+
+  @Override  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {    
+    PhysicalOperator leftPop = ((Prel) getLeft()).getPhysicalOperator(creator);
+
+    //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
+    if (leftPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
+      leftPop = new SelectionVectorRemover(leftPop);
+      creator.addPhysicalOperator(leftPop);
+    }
+
+    PhysicalOperator rightPop = ((Prel) getRight()).getPhysicalOperator(creator);
+
+    //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
+    if (rightPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
+      rightPop = new SelectionVectorRemover(rightPop);
+      creator.addPhysicalOperator(rightPop);
+    }
+    
+    JoinRelType jtype = this.getJoinType();
+    
+    final List<String> fields = getRowType().getFieldNames();
+    assert isUnique(fields);
+    final int leftCount = left.getRowType().getFieldCount();
+    final List<String> leftFields = fields.subList(0, leftCount);
+    final List<String> rightFields = fields.subList(leftCount, fields.size());
+    
+    List<JoinCondition> conditions = Lists.newArrayList();
+    
+    for (Pair<Integer, Integer> pair : Pair.zip(leftKeys, rightKeys)) {
+      conditions.add(new JoinCondition("==", new FieldReference(leftFields.get(pair.left)), new FieldReference(rightFields.get(pair.right))));
+    }
+    MergeJoinPOP mjoin = new MergeJoinPOP(leftPop, rightPop, conditions, jtype);
+    creator.addPhysicalOperator(mjoin);
+   
+    return mjoin;
+  }
+
+  public List<Integer> getLeftKeys() {
+    return this.leftKeys;
+  }
+  
+  public List<Integer> getRightKeys() {
+    return this.rightKeys;
+  }
+  
+//  public JoinCondition[] getJoinConditions() {
+//    return joinConditions;
+//  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
new file mode 100644
index 0000000..e693ec6
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrule.java
@@ -0,0 +1,101 @@
+/**
+ * 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.physical;
+
+import java.util.List;
+import java.util.logging.Logger;
+
+import net.hydromatic.optiq.util.BitSets;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.apache.drill.exec.planner.logical.DrillJoinRel;
+import org.apache.drill.exec.planner.logical.DrillJoinRule;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinRel;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelCollationImpl;
+import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.trace.EigenbaseTrace;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+
+public class MergeJoinPrule extends RelOptRule {
+  public static final RelOptRule INSTANCE = new MergeJoinPrule();
+  protected static final Logger tracer = EigenbaseTrace.getPlannerTracer();
+
+  private MergeJoinPrule() {
+    super(
+        RelOptHelper.some(DrillJoinRel.class, RelOptHelper.any(RelNode.class), RelOptHelper.any(RelNode.class)),
+        "Prel.MergeJoinPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final DrillJoinRel join = (DrillJoinRel) call.rel(0);
+    final RelNode left = call.rel(1);
+    final RelNode right = call.rel(2);
+
+    RelCollation collationLeft = getCollation(join.getLeftKeys());
+    RelCollation collationRight = getCollation(join.getRightKeys());
+    DrillDistributionTrait hashLeftPartition = new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(join.getLeftKeys())));
+    DrillDistributionTrait hashRightPartition = new DrillDistributionTrait(DrillDistributionTrait.DistributionType.HASH_DISTRIBUTED, ImmutableList.copyOf(getDistributionField(join.getRightKeys())));
+    
+    final RelTraitSet traitsLeft = left.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(collationLeft).plus(hashLeftPartition);   
+    final RelTraitSet traitsRight = right.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(collationRight).plus(hashRightPartition);
+    
+    final RelNode convertedLeft = convert(left, traitsLeft);
+    final RelNode convertedRight = convert(right, traitsRight);
+    
+    try {          
+      MergeJoinPrel newJoin = new MergeJoinPrel(join.getCluster(), traitsLeft, convertedLeft, convertedRight, join.getCondition(),
+                                                join.getJoinType());
+      call.transformTo(newJoin);
+    } catch (InvalidRelException e) {
+      tracer.warning(e.toString());
+    }
+
+  }
+  
+  private RelCollation getCollation(List<Integer> keys){    
+    List<RelFieldCollation> fields = Lists.newArrayList();
+    for (int key : keys) {
+      fields.add(new RelFieldCollation(key));
+    }
+    return RelCollationImpl.of(fields);
+  }
+
+  private List<DistributionField> getDistributionField(List<Integer> keys) {
+    List<DistributionField> distFields = Lists.newArrayList();
+
+    for (int key : keys) {
+      distFields.add(new DistributionField(key));
+    }
+     
+    return distFields;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
new file mode 100644
index 0000000..b1dacc5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/OrderedPartitionExchangePrel.java
@@ -0,0 +1,53 @@
+/**
+ * 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.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class OrderedPartitionExchangePrel extends SingleRel implements Prel {
+
+  public OrderedPartitionExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
+    super(cluster, traitSet, input);
+    assert input.getConvention() == Prel.DRILL_PHYSICAL;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+    //return planner.getCostFactory().makeCost(50, 50, 50);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new OrderedPartitionExchangePrel(getCluster(), traitSet, sole(inputs));
+  }
+  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    throw new IOException(this.getClass().getSimpleName() + " not supported yet!");
+  }
+  
+}


[39/51] [abbrv] git commit: Extract functions: avoid creating joda objects for each row

Posted by ja...@apache.org.
Extract functions: avoid creating joda objects for each row


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

Branch: refs/heads/master
Commit: ef8b59a2cc84bb8ce9ac474239db278abc9e70d9
Parents: 63b0da3
Author: vkorukanti <ve...@gmail.com>
Authored: Thu Apr 3 00:46:57 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../src/main/codegen/templates/Extract.java     | 99 ++++++++++----------
 1 file changed, 52 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ef8b59a2/exec/java-exec/src/main/codegen/templates/Extract.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/Extract.java b/exec/java-exec/src/main/codegen/templates/Extract.java
index d0e0afe..bef0614 100644
--- a/exec/java-exec/src/main/codegen/templates/Extract.java
+++ b/exec/java-exec/src/main/codegen/templates/Extract.java
@@ -31,8 +31,8 @@ import org.apache.drill.exec.record.RecordBatch;
 
 public class ${className} {
 
-<#list extract.toTypes as toUnit>
 <#list extract.fromTypes as fromUnit>
+<#list extract.toTypes as toUnit>
 <#if fromUnit == "Date" || fromUnit == "Time" || fromUnit == "TimeStamp">
   @FunctionTemplate(name = "extract${toUnit}", scope = FunctionTemplate.FunctionScope.SIMPLE,
       nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
@@ -40,23 +40,26 @@ public class ${className} {
 
     @Param ${fromUnit}Holder in;
     @Output BigIntHolder out;
+    @Workspace org.joda.time.MutableDateTime dateTime;
 
-    public void setup(RecordBatch incoming) { }
+    public void setup(RecordBatch incoming) {
+      dateTime = new org.joda.time.MutableDateTime(org.joda.time.DateTimeZone.UTC);
+    }
 
     public void eval() {
-      org.joda.time.MutableDateTime temp = new org.joda.time.MutableDateTime(in.value, org.joda.time.DateTimeZone.UTC);
+      dateTime.setMillis(in.value);
     <#if toUnit == "Second">
-      out.value = temp.getSecondOfMinute();
+      out.value = dateTime.getSecondOfMinute();
     <#elseif toUnit = "Minute">
-      out.value = temp.getMinuteOfHour();
+      out.value = dateTime.getMinuteOfHour();
     <#elseif toUnit = "Hour">
-      out.value = temp.getHourOfDay();
+      out.value = dateTime.getHourOfDay();
     <#elseif toUnit = "Day">
-      out.value = temp.getDayOfMonth();
+      out.value = dateTime.getDayOfMonth();
     <#elseif toUnit = "Month">
-      out.value = temp.getMonthOfYear();
+      out.value = dateTime.getMonthOfYear();
     <#elseif toUnit = "Year">
-      out.value = temp.getYear();
+      out.value = dateTime.getYear();
     </#if>
     }
   }
@@ -71,48 +74,50 @@ public class ${className} {
     public void setup(RecordBatch incoming) { }
 
     public void eval() {
-    <#if fromUnit == "Interval">
-
-      int years  = (in.months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      int months = (in.months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-
-      int millis = in.milliSeconds;
-
-      int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-
-      int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-
-      int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      org.joda.time.Period temp = new org.joda.time.Period(years, months, 0, in.days, hours, minutes, seconds, millis);
-
-    <#elseif fromUnit == "IntervalDay">
-
-      int millis = in.milliSeconds;
-
-      int hours  = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-      millis     = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
-
-      int minutes = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
-
-      int seconds = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      millis      = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
-      org.joda.time.Period temp = new org.joda.time.Period(0, 0, 0, in.days, hours, minutes, seconds, millis);
-
+  <#if fromUnit == "Interval">
+    <#if toUnit == "Year">
+      out.value = (in.months / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+    <#elseif toUnit == "Month">
+      out.value = (in.months % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+    <#elseif toUnit == "Day">
+      out.value = in.days;
+    <#elseif toUnit == "Hour">
+      out.value = in.milliSeconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+    <#elseif toUnit == "Minute">
+      int millis = in.milliSeconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+    <#elseif toUnit == "Second">
+      int millis = in.milliSeconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+    </#if>
+  <#elseif fromUnit == "IntervalDay">
+    <#if toUnit == "Year" || toUnit == "Month">
+      out.value = 0;
+    <#elseif toUnit == "Day">
+      out.value = in.days;
+    <#elseif toUnit == "Hour">
+      out.value = in.milliSeconds/(org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+    <#elseif toUnit == "Minute">
+      int millis = in.milliSeconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+    <#elseif toUnit == "Second">
+      int millis = in.milliSeconds % (org.apache.drill.exec.expr.fn.impl.DateUtility.hoursToMillis);
+      millis = millis % (org.apache.drill.exec.expr.fn.impl.DateUtility.minutesToMillis);
+      out.value = millis / (org.apache.drill.exec.expr.fn.impl.DateUtility.secondsToMillis);
+    </#if>
+  <#else> <#-- IntervalYear type -->
+    <#if toUnit == "Year">
+      out.value = (in.value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
+    <#elseif toUnit == "Month">
+      out.value = (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
     <#else>
-
-      int years  = (in.value / org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      int months = (in.value % org.apache.drill.exec.expr.fn.impl.DateUtility.yearsToMonths);
-      org.joda.time.Period temp = new org.joda.time.Period(years, months, 0, 0, 0, 0, 0, 0);
-
+      out.value = 0;
     </#if>
-      out.value = temp.get${toUnit}s();
+  </#if>
     }
   }
-  </#if>
+</#if>
 </#list>
 </#list>
 }
\ No newline at end of file


[19/51] [abbrv] git commit: Expression string builder bug, JODA cannot handle white space

Posted by ja...@apache.org.
Expression string builder bug, JODA cannot handle white space


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

Branch: refs/heads/master
Commit: 2faf6ef0253d6d3a8e3ce4d34ccc96c5a8590a74
Parents: 8c51248
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 16:19:49 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../apache/drill/common/expression/ExpressionStringBuilder.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2faf6ef0/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 35835c7..4bbc09a 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -136,7 +136,7 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
   public Void visitIntervalYearConstant(IntervalYearExpression lExpr, StringBuilder sb) throws RuntimeException {
     sb.append("cast( \"");
     sb.append(Period.years(lExpr.getIntervalYear()).toString());
-    sb.append(" \" as INTERVAL)");
+    sb.append("\" as INTERVAL)");
     return null;
   }
 
@@ -144,7 +144,7 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
   public Void visitIntervalDayConstant(IntervalDayExpression lExpr, StringBuilder sb) throws RuntimeException {
     sb.append("cast( \"");
     sb.append(Period.days(lExpr.getIntervalDay()).plusMillis(lExpr.getIntervalMillis()).toString());
-    sb.append(" \" as INTERVAL)");
+    sb.append("\" as INTERVAL)");
     return null;
   }
 


[21/51] [abbrv] git commit: bit agg functions

Posted by ja...@apache.org.
bit agg functions


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

Branch: refs/heads/master
Commit: 8e2b95057d9e46cc7cfed699eb93c47c386f2e6b
Parents: d91a01a
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 15:01:29 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/src/main/codegen/data/AggrTypes1.tdd    | 13 ++++++++++++-
 .../java/org/apache/drill/exec/vector/BitVector.java   | 12 ++++++++++++
 2 files changed, 24 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8e2b9505/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
index ac5d6eb..e0a1c11 100644
--- a/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
+++ b/exec/java-exec/src/main/codegen/data/AggrTypes1.tdd
@@ -17,10 +17,12 @@
 {
   aggrtypes: [
     {className: "Min", funcName: "min", types: [
+      {inputType: "Bit", outputType: "Bit", runningType: "Bit"},
       {inputType: "Int", outputType: "Int", runningType: "Int"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
+      {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
       {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
-      {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},    
+      {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "Float4", runningType: "Float4"},
       {inputType: "Float8", outputType: "Float8", runningType: "Float8"},
       {inputType: "NullableFloat4", outputType: "Float4", runningType: "Float4"},
@@ -28,8 +30,11 @@
      ]
    },
    {className: "Max", funcName: "max", types: [
+      {inputType: "Bit", outputType: "Bit", runningType: "Bit"},
       {inputType: "Int", outputType: "Int", runningType: "Int"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
+      {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
+      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "Float4", runningType: "Float4"},
@@ -39,8 +44,11 @@
      ]
    },
    {className: "Sum", funcName: "sum", types: [   
+      {inputType: "Bit", outputType: "Bit", runningType: "Bit"},
       {inputType: "Int", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
+      {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
+      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "Float8", runningType: "Float8"},
@@ -50,8 +58,11 @@
      ]
    }, 
    {className: "Count", funcName: "count", types: [
+      {inputType: "Bit", outputType: "Bit", runningType: "Bit"},
       {inputType: "Int", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "BigInt", outputType: "BigInt", runningType: "BigInt"},
+      {inputType: "NullableBit", outputType: "Bit", runningType: "Bit"},
+      {inputType: "NullableInt", outputType: "Int", runningType: "Int"},
       {inputType: "NullableInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "NullableBigInt", outputType: "BigInt", runningType: "BigInt"},
       {inputType: "Float4", outputType: "BigInt", runningType: "BigInt"},

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8e2b9505/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
index 1b4c3da..619fdad 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/vector/BitVector.java
@@ -242,6 +242,18 @@ public final class BitVector extends BaseDataValueVector implements FixedWidthVe
       return true;
     }
 
+    public boolean setSafe(int index, BitHolder holder) {
+      if(index >= getValueCapacity()) return false;
+      set(index, holder.value);
+      return true;
+    }
+
+    public boolean setSafe(int index, NullableBitHolder holder) {
+      if(index >= getValueCapacity()) return false;
+      set(index, holder.value);
+      return true;
+    }
+
     public final void setValueCount(int valueCount) {
       BitVector.this.valueCount = valueCount;
       data.writerIndex(getSizeFromCount(valueCount));


[14/51] [abbrv] git commit: Build the IfExpression correcty in ExpressionStringBuilder Use IntExpression instead of LongExpression for integers

Posted by ja...@apache.org.
Build the IfExpression correcty in ExpressionStringBuilder
Use IntExpression instead of LongExpression for integers


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

Branch: refs/heads/master
Commit: db0ff6364ee3ff27f6fe4a9a506617ee47c88ae5
Parents: 6976f92
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Mar 31 20:41:06 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../expression/ExpressionStringBuilder.java     |  3 +++
 .../common/expression/ValueExpressions.java     |  8 ++++++-
 .../sig/ConstantExpressionIdentifier.java       |  1 -
 .../drill/exec/expr/EvaluationVisitor.java      | 25 +++++++++++++++++++-
 .../drill/exec/fn/impl/TestMathFunctions.java   |  5 ++--
 5 files changed, 37 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/db0ff636/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 4bbc09a..16b3f4d 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -75,6 +75,9 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
       c.expression.accept(this, sb);
       sb.append(" ) ");
     }
+    sb.append(" else (");
+    ifExpr.elseExpression.accept(this, sb);
+    sb.append(" ) ");
     sb.append(" end ");
     sb.append(" ) ");
     return null;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/db0ff636/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 10cf152..4b83e7d 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -35,7 +35,7 @@ public class ValueExpressions {
   }
   
   public static LogicalExpression getInt(int i){
-    return new LongExpression(i);
+    return new IntExpression(i, ExpressionPosition.UNKNOWN);
   }
   
   public static LogicalExpression getFloat8(double d){
@@ -76,6 +76,12 @@ public class ValueExpressions {
 
   public static LogicalExpression getNumericExpression(String s, ExpressionPosition ep) {
     try {
+        int a = Integer.parseInt(s);
+        return new IntExpression(a, ep);
+    } catch (Exception e) {
+
+    }
+    try {
       long l = Long.parseLong(s);
       return new LongExpression(l, ep);
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/db0ff636/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
index b9a9ae2..2a87bab 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/sig/ConstantExpressionIdentifier.java
@@ -114,7 +114,6 @@ public class ConstantExpressionIdentifier implements ExprVisitor<Boolean, Identi
 
   @Override
   public Boolean visitIntConstant(ValueExpressions.IntExpression intExpr, IdentityHashMap<LogicalExpression, Object> value) throws RuntimeException {
-    value.put(intExpr, true);
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/db0ff636/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
index ac536e4..b7670ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/EvaluationVisitor.java
@@ -32,6 +32,7 @@ import org.apache.drill.common.expression.ValueExpressions;
 import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
 import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
 import org.apache.drill.common.expression.ValueExpressions.LongExpression;
+import org.apache.drill.common.expression.ValueExpressions.IntExpression;
 import org.apache.drill.common.expression.ValueExpressions.DateExpression;
 import org.apache.drill.common.expression.ValueExpressions.IntervalYearExpression;
 import org.apache.drill.common.expression.ValueExpressions.IntervalDayExpression;
@@ -140,7 +141,7 @@ public class EvaluationVisitor {
           if (HoldingContainer.isOptional()) {
             jc = conditionalBlock._if(HoldingContainer.getIsSet().cand(HoldingContainer.getValue()));
           } else {
-            jc = conditionalBlock._if(HoldingContainer.getValue());
+            jc = conditionalBlock._if(HoldingContainer.getValue().eq(JExpr.lit(1)));
           }
         } else {
           if (HoldingContainer.isOptional()) {
@@ -190,6 +191,13 @@ public class EvaluationVisitor {
     }
 
     @Override
+    public HoldingContainer visitIntConstant(IntExpression e, ClassGenerator<?> generator) throws RuntimeException {
+      HoldingContainer out = generator.declare(e.getMajorType());
+      generator.getEvalBlock().assign(out.getValue(), JExpr.lit(e.getInt()));
+      return out;
+    }
+
+    @Override
     public HoldingContainer visitDateConstant(DateExpression e, ClassGenerator<?> generator) throws RuntimeException {
       HoldingContainer out = generator.declare(e.getMajorType());
       generator.getEvalBlock().assign(out.getValue(), JExpr.lit(e.getDate()));
@@ -461,6 +469,21 @@ public class EvaluationVisitor {
     }
 
     @Override
+    public HoldingContainer visitIntConstant(IntExpression e, ClassGenerator<?> generator) throws RuntimeException {
+      if (constantBoundaries.contains(e)) {
+        generator.getMappingSet().enterConstant();
+        HoldingContainer c = super.visitIntConstant(e, generator);
+        //generator.getMappingSet().exitConstant();
+        //return c;
+        return renderConstantExpression(generator, c);
+      } else if (generator.getMappingSet().isWithinConstant()) {
+        return super.visitIntConstant(e, generator).setConstant(true);
+      } else {
+        return super.visitIntConstant(e, generator);
+      }
+    }
+
+    @Override
     public HoldingContainer visitDateConstant(DateExpression e, ClassGenerator<?> generator) throws RuntimeException {
       if (constantBoundaries.contains(e)) {
         generator.getMappingSet().enterConstant();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/db0ff636/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
index cc90f13..a23b31c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestMathFunctions.java
@@ -44,6 +44,7 @@ import org.apache.drill.exec.proto.ExecProtos.FragmentHandle;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.vector.BigIntVector;
+import org.apache.drill.exec.vector.IntVector;
 import org.apache.drill.exec.vector.Float8Vector;
 
 import org.junit.Test;
@@ -77,9 +78,9 @@ public class TestMathFunctions {
         SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
 
         while(exec.next()) {
-            BigIntVector intMulVector = exec.getValueVectorById(new SchemaPath("INTMUL", ExpressionPosition.UNKNOWN), BigIntVector.class);
+            IntVector intMulVector = exec.getValueVectorById(new SchemaPath("INTMUL", ExpressionPosition.UNKNOWN), IntVector.class);
             Float8Vector floatMulVector = exec.getValueVectorById(new SchemaPath("FLOATMUL", ExpressionPosition.UNKNOWN), Float8Vector.class);
-            BigIntVector intAddVector = exec.getValueVectorById(new SchemaPath("INTADD", ExpressionPosition.UNKNOWN), BigIntVector.class);
+            IntVector intAddVector = exec.getValueVectorById(new SchemaPath("INTADD", ExpressionPosition.UNKNOWN), IntVector.class);
             Float8Vector floatAddVector = exec.getValueVectorById(new SchemaPath("FLOATADD", ExpressionPosition.UNKNOWN), Float8Vector.class);
             assertEquals(exec.getRecordCount(), 1);
             assertEquals(intMulVector.getAccessor().get(0), 2);


[26/51] [abbrv] git commit: DRILL-437: Add support for POSTFIX operators. 1. IS NULL/IS NOT NULL 2. IS TRUE/IS FALSE 3. IS NOT TRUE/IS NOT FALSE

Posted by ja...@apache.org.
DRILL-437: Add support for POSTFIX operators.
1. IS NULL/IS NOT NULL
2. IS TRUE/IS FALSE
3. IS NOT TRUE/IS NOT FALSE


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

Branch: refs/heads/master
Commit: d3c7abeb556cbb91af81746ed9397af366e5cfd3
Parents: f48d998
Author: vkorukanti <ve...@gmail.com>
Authored: Tue Apr 1 12:08:20 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../main/codegen/templates/NullOperator.java    | 75 ++++++++++++++++++++
 .../apache/drill/exec/expr/fn/impl/IsFalse.java | 57 +++++++++++++++
 .../drill/exec/expr/fn/impl/IsNotFalse.java     | 57 +++++++++++++++
 .../drill/exec/expr/fn/impl/IsNotNull.java      | 40 -----------
 .../drill/exec/expr/fn/impl/IsNotTrue.java      | 56 +++++++++++++++
 .../apache/drill/exec/expr/fn/impl/IsNull.java  | 40 -----------
 .../apache/drill/exec/expr/fn/impl/IsTrue.java  | 56 +++++++++++++++
 .../drill/exec/planner/logical/DrillOptiq.java  | 11 +++
 .../drill/exec/resolver/TypeCastRules.java      |  8 ++-
 9 files changed, 317 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/codegen/templates/NullOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/NullOperator.java b/exec/java-exec/src/main/codegen/templates/NullOperator.java
new file mode 100644
index 0000000..054294b
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/NullOperator.java
@@ -0,0 +1,75 @@
+/**
+ * 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.
+ */
+<@pp.dropOutputFile />
+<#list vv.modes as mode>
+<#list vv.types as type>
+<#list type.minor as minor>
+
+<#assign className="GNullOp${mode.prefix}${minor.class}Holder" />
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${className}.java" />
+
+<#include "/@includes/license.ftl" />
+
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class ${className} {
+
+  @FunctionTemplate(names = {"isNull", "isnull", "is null"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class IsNull implements DrillSimpleFunc {
+
+    @Param ${mode.prefix}${minor.class}Holder input;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+    <#if mode.name == "Optional">
+      out.value = (input.isSet == 0 ? 1 : 0);
+    <#else>
+      out.value = 0;
+    </#if>
+    }
+  }
+
+  @FunctionTemplate(names = {"isNotNull", "isnotnull", "is not null"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class IsNotNull implements DrillSimpleFunc {
+
+    @Param ${mode.prefix}${minor.class}Holder input;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+    <#if mode.name == "Optional">
+      out.value = (input.isSet == 0 ? 0 : 1);
+    <#else>
+      out.value = 1;
+    </#if>
+    }
+  }
+}
+
+</#list>
+</#list>
+</#list>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsFalse.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsFalse.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsFalse.java
new file mode 100644
index 0000000..72db649
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsFalse.java
@@ -0,0 +1,57 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class IsFalse {
+
+  @FunctionTemplate(names = {"isFalse", "isfalse", "is false"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Optional implements DrillSimpleFunc {
+
+    @Param NullableBitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      if (in.isSet == 0)
+        out.value = 0;
+      else
+        out.value = (in.value == 0 ? 1 : 0);
+    }
+  }
+
+  @FunctionTemplate(names = {"isFalse", "isfalse", "is false"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Required implements DrillSimpleFunc {
+
+    @Param BitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = in.value == 0 ? 1 : 0;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotFalse.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotFalse.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotFalse.java
new file mode 100644
index 0000000..7fe5f2b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotFalse.java
@@ -0,0 +1,57 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class IsNotFalse {
+
+  @FunctionTemplate(names = {"isNotFalse", "isnotfalse", "is not false"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Optional implements DrillSimpleFunc {
+
+    @Param NullableBitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      if (in.isSet == 0)
+        out.value = 1;
+      else
+        out.value = in.value;
+    }
+  }
+
+  @FunctionTemplate(names = {"isNotFalse", "isnotfalse", "is not false"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Required implements DrillSimpleFunc {
+
+    @Param BitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotNull.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotNull.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotNull.java
deleted file mode 100644
index ad7a93e..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotNull.java
+++ /dev/null
@@ -1,40 +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.expr.fn.impl;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.BitHolder;
-import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
-import org.apache.drill.exec.record.RecordBatch;
-
-@FunctionTemplate(names = {"isNotNull", "isnotnull"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
-public class IsNotNull implements DrillSimpleFunc {
-
-  @Param NullableFloat8Holder input;
-  @Output BitHolder out;
-
-  public void setup(RecordBatch incoming) { }
-
-  public void eval() {
-    out.value = (input.isSet == 0 ? 0 : 1);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotTrue.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotTrue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotTrue.java
new file mode 100644
index 0000000..5e0bf48
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNotTrue.java
@@ -0,0 +1,56 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class IsNotTrue {
+
+  @FunctionTemplate(names = {"isNotTrue", "isnottrue", "is not true"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Optional implements DrillSimpleFunc {
+
+    @Param NullableBitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      if (in.isSet == 0)
+        out.value = 1;
+      else
+        out.value = (in.value == 0 ? 1 : 0);
+    }
+  }
+
+  @FunctionTemplate(names = {"isNotTrue", "isnottrue", "is not true"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Required implements DrillSimpleFunc {
+
+    @Param BitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = in.value == 0 ? 1 : 0;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNull.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNull.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNull.java
deleted file mode 100644
index 02905d4..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsNull.java
+++ /dev/null
@@ -1,40 +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.expr.fn.impl;
-
-import org.apache.drill.exec.expr.DrillSimpleFunc;
-import org.apache.drill.exec.expr.annotations.FunctionTemplate;
-import org.apache.drill.exec.expr.annotations.Output;
-import org.apache.drill.exec.expr.annotations.Param;
-import org.apache.drill.exec.expr.holders.BitHolder;
-import org.apache.drill.exec.expr.holders.NullableFloat8Holder;
-import org.apache.drill.exec.record.RecordBatch;
-
-@FunctionTemplate(names = {"isNull", "isnull"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
-public class IsNull implements DrillSimpleFunc {
-
-  @Param NullableFloat8Holder input;
-  @Output BitHolder out;
-
-  public void setup(RecordBatch incoming) { }
-
-  public void eval() {
-    out.value = (input.isSet == 0 ? 1 : 0);
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsTrue.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsTrue.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsTrue.java
new file mode 100644
index 0000000..d85a807
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/IsTrue.java
@@ -0,0 +1,56 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.*;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class IsTrue {
+
+  @FunctionTemplate(names = {"isTrue", "istrue", "is true"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Optional implements DrillSimpleFunc {
+
+    @Param NullableBitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      if (in.isSet == 0)
+        out.value = 0;
+      else
+        out.value = in.value;
+    }
+  }
+
+  @FunctionTemplate(names = {"isTrue", "istrue", "is true"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Required implements DrillSimpleFunc {
+
+    @Param BitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = in.value;
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 4926d7b..ad6aa3c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -105,6 +105,17 @@ public class DrillOptiq {
       case FUNCTION:
         logger.debug("Function");
         return getDrillFunctionFromOptiqCall(call);
+      case POSTFIX:
+        logger.debug("Postfix");
+        switch(call.getKind()){
+        case IS_NULL:
+        case IS_TRUE:
+        case IS_FALSE:
+        case OTHER:
+          return FunctionCallFactory.createExpression(call.getOperator().getName().toLowerCase(),
+              ExpressionPosition.UNKNOWN, call.getOperands().get(0).accept(this));
+        }
+        throw new AssertionError("todo: implement syntax " + syntax + "(" + call + ")");
       case PREFIX:
         logger.debug("Prefix");
         LogicalExpression arg = call.getOperands().get(0).accept(this);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d3c7abeb/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
index ed89301..67769c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
@@ -565,8 +565,10 @@ public class TypeCastRules {
     rules.put(MinorType.VARBINARY, rule);
   }
 
-  public static boolean isCastable(MajorType from, MajorType to) {
-    return from.getMinorType().equals(MinorType.NULL) ||      //null could be casted to any other type. 
+  public static boolean isCastable(MajorType from, MajorType to, NullHandling nullHandling) {
+    if (nullHandling == NullHandling.INTERNAL && from.getMode() != to.getMode()) return false;
+
+    return from.getMinorType().equals(MinorType.NULL) ||      //null could be casted to any other type.
            (rules.get(to.getMinorType()) == null ? false : rules.get(to.getMinorType()).contains(from.getMinorType()));
   }
 
@@ -586,7 +588,7 @@ public class TypeCastRules {
       MajorType argType = call.args.get(i).getMajorType();
       MajorType parmType = holder.getParmMajorType(i);
 
-      if (!TypeCastRules.isCastable(argType, parmType)) {
+      if (!TypeCastRules.isCastable(argType, parmType, holder.getNullHandling())) {
         return -1;
       }
 


[05/51] [abbrv] DRILL-450: Add exchange rules, move from BasicOptimizer to Optiq

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
index 20609b8..3981ee1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaDrillTable.java
@@ -27,8 +27,8 @@ public class InfoSchemaDrillTable extends DrillTable{
 
   private final SelectedTable table;
   
-  public InfoSchemaDrillTable(String storageEngineName, SelectedTable selection, StoragePluginConfig storageEngineConfig) {
-    super(storageEngineName, selection, storageEngineConfig);
+  public InfoSchemaDrillTable(InfoSchemaStoragePlugin plugin, String storageEngineName, SelectedTable selection, StoragePluginConfig storageEngineConfig) {
+    super(storageEngineName, plugin, selection);
     this.table = selection;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
index 891419d..a7d7c5c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/ischema/InfoSchemaStoragePlugin.java
@@ -24,6 +24,8 @@ import java.util.Set;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.common.logical.data.Scan;
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -53,25 +55,30 @@ public class InfoSchemaStoragePlugin extends AbstractStoragePlugin{
   }
 
   @Override
-  public InfoSchemaGroupScan getPhysicalScan(Scan scan) throws IOException {
-    SelectedTable table = scan.getSelection().getWith(context.getConfig(),  SelectedTable.class);
+  public InfoSchemaGroupScan getPhysicalScan(JSONOptions selection) throws IOException {
+    SelectedTable table = selection.getWith(context.getConfig(),  SelectedTable.class);
     return new InfoSchemaGroupScan(table);
   }
 
   @Override
+  public StoragePluginConfig getConfig() {
+    return this.config;
+  }
+  
+  @Override
   public Schema createAndAddSchema(SchemaPlus parent) {
-    Schema s = new ISchema(parent);
+    Schema s = new ISchema(parent, this);
     parent.add(s);
     return s;
   }
   
   private class ISchema extends AbstractSchema{
     private Map<String, InfoSchemaDrillTable> tables;
-    public ISchema(SchemaPlus parent){
+    public ISchema(SchemaPlus parent, InfoSchemaStoragePlugin plugin){
       super(new SchemaHolder(parent), "INFORMATION_SCHEMA");
       Map<String, InfoSchemaDrillTable> tbls = Maps.newHashMap();
       for(SelectedTable tbl : SelectedTable.values()){
-        tbls.put(tbl.name(), new InfoSchemaDrillTable("INFORMATION_SCHEMA", tbl, config));  
+        tbls.put(tbl.name(), new InfoSchemaDrillTable(plugin, "INFORMATION_SCHEMA", tbl, config));  
       }
       this.tables = ImmutableMap.copyOf(tbls);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
index 47218fe..c20c134 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/ParquetGroupScan.java
@@ -96,10 +96,15 @@ public class ParquetGroupScan extends AbstractGroupScan {
   }
 
   @JsonProperty("format")
-  public ParquetFormatConfig getEngineConfig() {
+  public ParquetFormatConfig getFormatConfig() {
     return this.formatConfig;
   }
 
+  @JsonProperty("storage")
+  public StoragePluginConfig getEngineConfig() {
+    return this.formatPlugin.getStorageConfig();
+  }
+
   @JsonCreator
   public ParquetGroupScan( //
       @JsonProperty("entries") List<ReadEntryWithPath> entries, //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 90b2a4d..5d53b7d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -346,6 +346,16 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
 
   private void runSQL(String sql) {
     try{
+      DrillSqlWorker sqlWorker = new DrillSqlWorker(context.getFactory(), context.getFunctionRegistry());
+      
+      PhysicalPlan physical = sqlWorker.getPhysicalPlan(sql, context);
+      
+      if(logger.isDebugEnabled()) {
+        logger.debug("Distributed Physical {}", context.getConfig().getMapper().writeValueAsString(physical));
+        System.out.println(context.getConfig().getMapper().writeValueAsString(physical));
+      }
+      
+      runPhysicalPlan(physical);
     }catch(Exception e){
       fail("Failure while parsing sql.", e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
index eaf921d..b305d0d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/QueryManager.java
@@ -56,7 +56,7 @@ import com.google.common.collect.Maps;
 /**
  * Each Foreman holds its own fragment manager.  This manages the events associated with execution of a particular query across all fragments.  
  */
-class QueryManager implements FragmentStatusListener{
+public class QueryManager implements FragmentStatusListener{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryManager.class);
   
   public Map<FragmentHandle, FragmentData> map = Maps.newHashMap(); // doesn't need to be thread safe as map is generated in a single thread and then accessed by multiple threads for reads only.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/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 7199c65..5baaf63 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
@@ -19,23 +19,21 @@ package org.apache.drill;
 
 import org.apache.drill.common.util.TestTools;
 import org.apache.drill.exec.client.QuerySubmitter;
+import org.junit.Ignore;
 import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
 
-public class TestExampleQueries extends BaseTestQuery{
+public class TestExampleQueries {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExampleQueries.class);
   
-  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(20000);
+  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(10000000);
   
   @Test
   public void testSelectWithLimit() throws Exception{
     test("select * from cp.`employee.json` limit 5");
   }
   
-
-
-        
   @Test
   public void testJoin() throws Exception{
     test("SELECT\n" + 
@@ -59,4 +57,17 @@ public class TestExampleQueries extends BaseTestQuery{
     test("select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
   }
   
+  private void test(String sql) throws Exception{
+    boolean good = false;
+    sql = sql.replace("[WORKING_PATH]", TestTools.getWorkingPath());
+    
+    try{
+      QuerySubmitter s = new QuerySubmitter();
+      s.submitQuery(null, sql, "sql", null, true, 1, "tsv");
+      good = true;
+    }finally{
+      if(!good) Thread.sleep(2000);
+    }
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
index 346ed56..8643fa3 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
@@ -24,6 +24,7 @@ public class TestTpchQueries extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchQueries.class);
   
   @Test
+  @Ignore
   public void tpch01() throws Exception{
     testSqlFromFile("queries/tpch/01.sql");
   }
@@ -53,6 +54,7 @@ public class TestTpchQueries extends BaseTestQuery{
   }
   
   @Test  // DRILL-356
+  @Ignore
   public void tpch06() throws Exception{
     testSqlFromFile("queries/tpch/06.sql");
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 6e681e1..8e92181 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -28,6 +28,7 @@ import mockit.NonStrictExpectations;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.FileUtils;
+import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
@@ -38,12 +39,17 @@ import org.apache.drill.exec.physical.impl.OperatorCreatorRegistry;
 import org.apache.drill.exec.physical.impl.ImplCreator;
 import org.apache.drill.exec.physical.impl.SimpleRootExec;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
+import org.apache.drill.exec.pop.PopUnitTestBase;
 import org.apache.drill.exec.proto.CoordinationProtos;
 import org.apache.drill.exec.proto.ExecProtos;
+import org.apache.drill.exec.proto.UserProtos;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.rpc.user.UserServer;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.AfterClass;
@@ -55,7 +61,7 @@ import com.google.common.io.Files;
 import com.codahale.metrics.MetricRegistry;
 
 
-public class TestMergeJoin {
+public class TestMergeJoin extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoin.class);
 
   DrillConfig c = DrillConfig.create();
@@ -292,12 +298,35 @@ public class TestMergeJoin {
 
   }
 
+  @Test
+  public void testMergeJoinEmptyBatch() throws Exception {
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);      
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
+
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+          Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
+              Charsets.UTF_8));
+      int count = 0;
+      for(QueryResultBatch b : results) {
+        if (b.getHeader().getRowCount() != 0)
+          count += b.getHeader().getRowCount();
+      }
+      assertEquals(0, count);
+    }
+  }  
+  
+  
   @AfterClass
   public static void tearDown() throws Exception{
     // pause to get logger to catch up.
     Thread.sleep(1000);
   }
 
+  
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
deleted file mode 100644
index 68d1eef..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/planner/physical/TestPhysicalPlanning.java
+++ /dev/null
@@ -1,39 +0,0 @@
-package org.apache.drill.exec.planner.physical;
-
-import mockit.NonStrictExpectations;
-
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.expression.FunctionRegistry;
-import org.apache.drill.exec.memory.TopLevelAllocator;
-import org.apache.drill.exec.planner.sql.DrillSqlWorker;
-import org.apache.drill.exec.server.DrillbitContext;
-import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.junit.Test;
-
-import com.codahale.metrics.MetricRegistry;
-
-public class TestPhysicalPlanning {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestPhysicalPlanning.class);
-  
-  @Test
-  public void testSimpleQuery(final DrillbitContext bitContext) throws Exception{
-    
-    final DrillConfig c = DrillConfig.create();
-    new NonStrictExpectations() {
-      {
-        bitContext.getMetrics();
-        result = new MetricRegistry();
-        bitContext.getAllocator();
-        result = new TopLevelAllocator();
-        bitContext.getConfig();
-        result = c;
-      }
-    };
-    
-    FunctionRegistry reg = new FunctionRegistry(c);
-    StoragePluginRegistry registry = new StoragePluginRegistry(bitContext);
-    DrillSqlWorker worker = new DrillSqlWorker(registry.getSchemaFactory(), reg);
-    worker.getPhysicalPlan("select * from cp.`employee.json`");
-    
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
index b5ad235..9020c1a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
@@ -399,7 +399,7 @@ public class JSONRecordReaderTest {
 
     JSONRecordReader jr = new JSONRecordReader(context,
         FileUtils.getResourceAsFile("/scan_json_test_7.json").toURI().toString(),
-        FileSystem.getLocal(new Configuration()), null, null);
+        FileSystem.getLocal(new Configuration()), null);
 
     MockOutputMutator mutator = new MockOutputMutator();
     List<ValueVector> addFields = mutator.getAddFields();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index f6a8aa4..f6a7d97 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -429,8 +429,7 @@ public class ParquetRecordReaderTest {
     FileSystem fs = new CachedSingleFileSystem(fileName);
     for(int i = 0; i < 25; i++){
       ParquetRecordReader rr = new ParquetRecordReader(context, 256000, fileName, 0, fs,
-          new CodecFactoryExposer(dfsConfig), f.getParquetMetadata(), new FieldReference("_MAP",
-          ExpressionPosition.UNKNOWN), columns);
+          new CodecFactoryExposer(dfsConfig), f.getParquetMetadata(), columns);
       TestOutputMutator mutator = new TestOutputMutator();
       rr.setup(mutator);
       Stopwatch watch = new Stopwatch();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json b/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
new file mode 100644
index 0000000..549cb83
--- /dev/null
+++ b/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
@@ -0,0 +1,47 @@
+{
+  head:{
+    type:"APACHE_DRILL_PHYSICAL",
+    version:"1",
+    generator:{
+      type:"manual"
+    }
+  },
+  graph:[
+    {
+      @id:1,
+      pop:"mock-scan",
+      url: "http://source1.apache.org",
+      entries:[
+        {records: 0, types: [
+          {name: "blue", type: "INT", mode: "REQUIRED"},
+          {name: "red", type: "INT", mode: "REQUIRED"},
+          {name: "green", type: "INT", mode: "REQUIRED"}
+        ]}
+      ]
+    },
+    {
+      @id:2,
+      pop:"mock-scan",
+      url: "http://source2.apache.org",
+      entries:[
+        {records: 50, types: [
+          {name: "blue1", type: "INT", mode: "REQUIRED"},
+          {name: "red1", type: "INT", mode: "REQUIRED"},
+          {name: "green1", type: "INT", mode: "REQUIRED"}
+        ]}
+      ]
+    },
+    {
+      @id: 3,
+      right: 1,
+      left: 2,
+      pop: "merge-join",
+      join-conditions: [ {relationship: "==", left: "blue1", right: "blue"} ]
+    },
+    {
+      @id: 4,
+      child: 3,
+      pop: "screen"
+    }
+  ]
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 44eb1c1..97cf61d 100644
--- a/pom.xml
+++ b/pom.xml
@@ -594,7 +594,7 @@
           <dependency>
             <groupId>net.hydromatic</groupId>
             <artifactId>optiq-core</artifactId>
-            <version>0.4.18</version>
+            <version>0.6-SNAPSHOT</version>
             <exclusions>
               <exclusion>
                 <groupId>org.jgrapht</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/nationsMF/nation.parquet
----------------------------------------------------------------------
diff --git a/sample-data/nationsMF/nation.parquet b/sample-data/nationsMF/nation.parquet
new file mode 100644
index 0000000..db65492
Binary files /dev/null and b/sample-data/nationsMF/nation.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/nationsMF/nation2.parquet
----------------------------------------------------------------------
diff --git a/sample-data/nationsMF/nation2.parquet b/sample-data/nationsMF/nation2.parquet
new file mode 100644
index 0000000..db65492
Binary files /dev/null and b/sample-data/nationsMF/nation2.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/nationsSF/nation.parquet
----------------------------------------------------------------------
diff --git a/sample-data/nationsSF/nation.parquet b/sample-data/nationsSF/nation.parquet
new file mode 100644
index 0000000..db65492
Binary files /dev/null and b/sample-data/nationsSF/nation.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/regionsMF/region.parquet
----------------------------------------------------------------------
diff --git a/sample-data/regionsMF/region.parquet b/sample-data/regionsMF/region.parquet
new file mode 100644
index 0000000..ab8122c
Binary files /dev/null and b/sample-data/regionsMF/region.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/regionsMF/region2.parquet
----------------------------------------------------------------------
diff --git a/sample-data/regionsMF/region2.parquet b/sample-data/regionsMF/region2.parquet
new file mode 100644
index 0000000..ab8122c
Binary files /dev/null and b/sample-data/regionsMF/region2.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sample-data/regionsSF/region.parquet
----------------------------------------------------------------------
diff --git a/sample-data/regionsSF/region.parquet b/sample-data/regionsSF/region.parquet
new file mode 100644
index 0000000..ab8122c
Binary files /dev/null and b/sample-data/regionsSF/region.parquet differ

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/main/java/org/apache/drill/jdbc/ConnectionConfig.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/ConnectionConfig.java b/sqlparser/src/main/java/org/apache/drill/jdbc/ConnectionConfig.java
deleted file mode 100644
index 5a81035..0000000
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/ConnectionConfig.java
+++ /dev/null
@@ -1,41 +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.jdbc;
-
-import java.util.Properties;
-import java.util.TimeZone;
-
-public class ConnectionConfig {
-  private final Properties props;
-  
-  public ConnectionConfig(Properties p){
-    this.props = p;
-  }
-  
-  public boolean isLocal(){
-    return "local".equals(props.getProperty("zk"));
-  }
-  public String getZookeeperConnectionString(){
-    return props.getProperty("zk");
-  }
-  
-  public TimeZone getTimeZone(){
-    return TimeZone.getDefault(); 
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
index ab1259a..5434f3d 100644
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnection.java
@@ -35,7 +35,7 @@ public interface DrillConnection extends Connection{
   /** Returns a view onto this connection's configuration properties. Code
    * within Optiq should use this view rather than calling
    * {@link java.util.Properties#getProperty(String)}. */
-  ConnectionConfig config();
+  DrillConnectionConfig config();
   
   public DrillClient getClient();
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
new file mode 100644
index 0000000..0c0989d
--- /dev/null
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionConfig.java
@@ -0,0 +1,45 @@
+/**
+ * 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.util.Properties;
+import java.util.TimeZone;
+
+import net.hydromatic.avatica.ConnectionConfig;
+import net.hydromatic.avatica.ConnectionConfigImpl;
+
+public class DrillConnectionConfig extends ConnectionConfigImpl {
+  private final Properties props;
+  
+  public DrillConnectionConfig(Properties p){
+    super(p);
+    this.props = p;
+  }
+  
+  public boolean isLocal(){
+    return "local".equals(props.getProperty("zk"));
+  }
+  public String getZookeeperConnectionString(){
+    return props.getProperty("zk");
+  }
+  
+  public TimeZone getTimeZone(){
+    return TimeZone.getDefault(); 
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
index 11a1157..5dc0d58 100644
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillConnectionImpl.java
@@ -45,7 +45,7 @@ import org.apache.drill.exec.server.RemoteServiceSet;
  */
 abstract class DrillConnectionImpl extends AvaticaConnection implements org.apache.drill.jdbc.DrillConnection {
   public final DrillStatementRegistry registry = new DrillStatementRegistry();
-  final ConnectionConfig config;
+  final DrillConnectionConfig config;
   
   
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillConnection.class);
@@ -57,7 +57,7 @@ abstract class DrillConnectionImpl extends AvaticaConnection implements org.apac
   
   protected DrillConnectionImpl(Driver driver, AvaticaFactory factory, String url, Properties info)  throws SQLException{
     super(driver, factory, url, info);
-    this.config = new ConnectionConfig(info);
+    this.config = new DrillConnectionConfig(info);
   
   this.allocator = new TopLevelAllocator();
     
@@ -90,7 +90,7 @@ abstract class DrillConnectionImpl extends AvaticaConnection implements org.apac
   }
   
   
-  public ConnectionConfig config(){
+  public DrillConnectionConfig config(){
     return config;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
new file mode 100644
index 0000000..9977285
--- /dev/null
+++ b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
@@ -0,0 +1,218 @@
+/**
+ * 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.test;
+
+import java.nio.file.Paths;
+import java.sql.Connection;
+import java.sql.DriverManager;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.Statement;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.store.hive.HiveTestDataGenerator;
+import org.apache.drill.jdbc.Driver;
+import org.junit.BeforeClass;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+import com.google.common.base.Stopwatch;
+
+public class TestJdbcDistQuery {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJdbcDistQuery.class);
+
+  
+  // Set a timeout unless we're debugging.
+  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(50000);
+
+  private static final String WORKING_PATH;
+  static{
+    Driver.load();
+    WORKING_PATH = Paths.get("").toAbsolutePath().toString();
+    
+  }
+  
+  @BeforeClass
+  public static void generateHive() throws Exception{
+    new HiveTestDataGenerator().generateTestData();
+  }
+  
+
+  @Test 
+  public void testSimpleQuerySingleFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, R_NAME "
+        + "from dfs.`%s/../sample-data/regionsSF/`", WORKING_PATH));    
+  }
+
+  
+  @Test 
+  public void testSimpleQueryMultiFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, R_NAME "
+        + "from dfs.`%s/../sample-data/regionsMF/`", WORKING_PATH));    
+  }
+
+  @Test 
+  public void testWhereOverSFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, R_NAME "
+        + "from dfs.`%s/../sample-data/regionsSF/` "
+        + "WHERE R_REGIONKEY = 1", WORKING_PATH));    
+  }
+
+  @Test 
+  public void testWhereOverMFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, R_NAME "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "WHERE R_REGIONKEY = 1", WORKING_PATH));    
+  }
+
+  
+  @Test 
+  public void testAggSingleFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsSF/` "
+        + "group by R_REGIONKEY", WORKING_PATH));    
+  }
+  
+  @Test
+  public void testAggMultiFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "group by R_REGIONKEY", WORKING_PATH));    
+  }
+ 
+  @Test
+  public void testAggOrderByDiffGKeyMultiFile() throws Exception{    
+    testQuery(String.format("select R_REGIONKEY, SUM(cast(R_REGIONKEY AS int)) As S "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "group by R_REGIONKEY ORDER BY S", WORKING_PATH));    
+  }
+ 
+  @Test
+  public void testAggOrderBySameGKeyMultiFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, SUM(cast(R_REGIONKEY AS int)) As S "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "group by R_REGIONKEY "
+        + "ORDER BY R_REGIONKEY", WORKING_PATH));   
+  }
+   
+  @Test
+  public void testJoinSingleFile() throws Exception{
+    testQuery(String.format("select T1.R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsSF/` as T1 "
+        + "join dfs.`%s/../sample-data/nationsSF/` as T2 "
+        + "on T1.R_REGIONKEY = T2.N_REGIONKEY", WORKING_PATH, WORKING_PATH));    
+  }
+
+  @Test
+  public void testJoinMultiFile() throws Exception{
+    testQuery(String.format("select T1.R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsMF/` as T1 "
+        + "join dfs.`%s/../sample-data/nationsMF/` as T2 "
+        + "on T1.R_REGIONKEY = T2.N_REGIONKEY", WORKING_PATH, WORKING_PATH));     
+  }
+
+  @Test
+  public void testJoinMFileWhere() throws Exception{
+    testQuery(String.format("select T1.R_REGIONKEY, T1.R_NAME "
+        + "from dfs.`%s/../sample-data/regionsMF/` as T1 "
+        + "join dfs.`%s/../sample-data/nationsMF/` as T2 "
+        + "on T1.R_REGIONKEY = T2.N_REGIONKEY "
+        + "WHERE T1.R_REGIONKEY  = 3 ", WORKING_PATH, WORKING_PATH));     
+  }
+
+  @Test
+  public void testSortSingleFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsSF/` "
+        + "order by R_REGIONKEY", WORKING_PATH));   
+  }
+
+  @Test
+  public void testSortMultiFile() throws Exception{
+    testQuery(String.format("select R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "order by R_REGIONKEY", WORKING_PATH));   
+  }
+
+  @Test
+  public void testSortMFileWhere() throws Exception{
+    testQuery(String.format("select R_REGIONKEY "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "WHERE R_REGIONKEY = 1 "
+        + "order by R_REGIONKEY ", WORKING_PATH ));   
+  }
+ 
+  @Test
+  public void testJoinAggSortWhere() throws Exception{
+    testQuery(String.format("select T1.R_REGIONKEY, COUNT(1) as CNT "
+        + "from dfs.`%s/../sample-data/regionsMF/` as T1 "
+        + "join dfs.`%s/../sample-data/nationsMF/` as T2 "
+        + "on T1.R_REGIONKEY = T2.N_REGIONKEY "
+        + "WHERE T1.R_REGIONKEY  = 3 "
+        + "GROUP BY T1.R_REGIONKEY "
+        + "ORDER BY T1.R_REGIONKEY",WORKING_PATH, WORKING_PATH ));     
+  }
+  
+  @Test
+  public void testSelectLimit() throws Exception{
+    testQuery(String.format("select R_REGIONKEY, R_NAME "
+        + "from dfs.`%s/../sample-data/regionsMF/` "
+        + "limit 2", WORKING_PATH));   
+  }
+  
+ private void testQuery(String sql) throws Exception{
+    boolean success = false;
+    try (Connection c = DriverManager.getConnection("jdbc:drill:zk=local", null);) {
+      for (int x = 0; x < 1; x++) {
+        Stopwatch watch = new Stopwatch().start();
+        Statement s = c.createStatement();
+        ResultSet r = s.executeQuery(sql);
+        boolean first = true;
+        while (r.next()) {
+          ResultSetMetaData md = r.getMetaData();
+          if (first == true) {
+            for (int i = 1; i <= md.getColumnCount(); i++) {
+              System.out.print(md.getColumnName(i));
+              System.out.print('\t');
+            }
+            System.out.println();
+            first = false;
+          }
+
+          for (int i = 1; i <= md.getColumnCount(); i++) {
+            System.out.print(r.getObject(i));
+            System.out.print('\t');
+          }
+          System.out.println();
+        }
+
+        System.out.println(String.format("Query completed in %d millis.", watch.elapsed(TimeUnit.MILLISECONDS)));
+      }
+
+      System.out.println("\n\n\n");
+      success = true;
+    }finally{
+      if(!success) Thread.sleep(2000);
+    }
+    
+    
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
index 36bbc51..c067ae0 100644
--- a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
+++ b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
@@ -41,7 +41,7 @@ public class TestJdbcQuery {
 
   
   // Set a timeout unless we're debugging.
-  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(20000);
+  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(200000000);
 
   private static final String WORKING_PATH;
   static{
@@ -56,6 +56,7 @@ public class TestJdbcQuery {
   }
   
   @Test
+  @Ignore
   public void testHiveRead() throws Exception{
     testQuery("select * from hive.kv");
   }
@@ -67,6 +68,7 @@ public class TestJdbcQuery {
   }
 
   @Test
+  @Ignore
   public void testJsonQuery() throws Exception{
     testQuery("select * from cp.`employee.json`");
   }
@@ -86,11 +88,13 @@ public class TestJdbcQuery {
   }
 
   @Test 
+  @Ignore
   public void testWorkspace() throws Exception{
     testQuery(String.format("select * from dfs.home.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
   @Test 
+  @Ignore
   public void testWildcard() throws Exception{
     testQuery(String.format("select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
@@ -106,16 +110,19 @@ public class TestJdbcQuery {
   }
 
   @Test 
+  @Ignore
   public void testLogicalExplain() throws Exception{
     testQuery(String.format("EXPLAIN PLAN WITHOUT IMPLEMENTATION FOR select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
   @Test 
+  @Ignore
   public void testPhysicalExplain() throws Exception{
     testQuery(String.format("EXPLAIN PLAN FOR select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
   
   @Test 
+  @Ignore
   public void checkUnknownColumn() throws Exception{
     testQuery(String.format("SELECT unknownColumn FROM dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }


[08/51] [abbrv] git commit: DRILL-450: Add exchange rules, move from BasicOptimizer to Optiq

Posted by ja...@apache.org.
DRILL-450: Add exchange rules, move from BasicOptimizer to Optiq


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

Branch: refs/heads/master
Commit: 6b517daad2a817231df9959f2f3e5c53bd728117
Parents: 22c4190
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Wed Feb 26 17:04:37 2014 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:08 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/parser/ExprLexer.g  |  27 +--
 .../apache/drill/common/logical/data/Order.java |   6 +-
 .../drill/exec/store/hbase/HBaseGroupScan.java  |  16 +-
 .../exec/store/hbase/HBaseSchemaFactory.java    |  15 +-
 .../exec/store/hbase/HBaseStoragePlugin.java    |  29 ++-
 .../drill/exec/store/hbase/HBaseSubScan.java    |  15 +-
 .../drill/exec/expr/fn/impl/HashFunctions.java  |  31 +++
 .../org/apache/drill/exec/ops/QueryContext.java |   4 +
 .../apache/drill/exec/opt/BasicOptimizer.java   |   6 +-
 .../drill/exec/physical/base/AbstractBase.java  |   6 +
 .../exec/physical/base/AbstractSubScan.java     |   6 +
 .../drill/exec/physical/base/GroupScan.java     |   3 +-
 .../exec/physical/base/PhysicalOperator.java    |   9 +
 .../drill/exec/physical/config/Filter.java      |  15 +-
 .../physical/config/HashToRandomExchange.java   |   5 +-
 .../exec/physical/config/MergeJoinPOP.java      |   1 -
 .../drill/exec/physical/config/Project.java     |   6 +-
 .../drill/exec/physical/config/Screen.java      |   1 +
 .../physical/config/SingleMergeExchange.java    |   5 +
 .../apache/drill/exec/physical/config/Sort.java |   6 +-
 .../exec/physical/impl/TopN/TopNBatch.java      |   2 +-
 .../exec/physical/impl/join/JoinStatus.java     |   4 +-
 .../exec/physical/impl/join/MergeJoinBatch.java |  18 +-
 .../physical/impl/limit/LimitRecordBatch.java   |   1 +
 .../impl/mergereceiver/MergingRecordBatch.java  |   4 +-
 .../OrderedPartitionRecordBatch.java            |   2 +-
 .../exec/physical/impl/sort/SortBatch.java      |   2 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |   4 +-
 .../exec/planner/common/BaseFilterRel.java      |  53 -----
 .../exec/planner/common/BaseProjectRel.java     |  68 ------
 .../drill/exec/planner/common/BaseScanRel.java  |  36 ---
 .../exec/planner/common/BaseScreenRel.java      |  41 ----
 .../planner/common/DrillAggregateRelBase.java   |  42 ++++
 .../exec/planner/common/DrillFilterRelBase.java |  55 +++++
 .../exec/planner/common/DrillJoinRelBase.java   |  80 +++++++
 .../exec/planner/common/DrillLimitRel.java      |  75 -------
 .../exec/planner/common/DrillLimitRelBase.java  |  57 +++++
 .../planner/common/DrillProjectRelBase.java     |  69 ++++++
 .../drill/exec/planner/common/DrillRelNode.java |  25 +++
 .../exec/planner/common/DrillScanRelBase.java   |  39 ++++
 .../exec/planner/common/DrillScreenRelBase.java |  43 ++++
 .../exec/planner/common/DrillStoreRel.java      |  47 ----
 .../exec/planner/common/DrillStoreRelBase.java  |  42 ++++
 .../exec/planner/common/DrillUnionRel.java      |  73 -------
 .../exec/planner/common/DrillUnionRelBase.java  |  37 ++++
 .../exec/planner/logical/DrillAggregateRel.java |   8 +-
 .../exec/planner/logical/DrillFilterRel.java    |   4 +-
 .../exec/planner/logical/DrillJoinRel.java      |  28 +--
 .../exec/planner/logical/DrillLimitRel.java     |  10 +-
 .../exec/planner/logical/DrillLimitRule.java    |   1 -
 .../exec/planner/logical/DrillProjectRel.java   |  11 +-
 .../drill/exec/planner/logical/DrillRel.java    |   6 +-
 .../exec/planner/logical/DrillRuleSets.java     |  49 +++--
 .../exec/planner/logical/DrillScanRel.java      |  10 +-
 .../exec/planner/logical/DrillScanRule.java     |   2 +-
 .../exec/planner/logical/DrillScreenRel.java    |   4 +-
 .../exec/planner/logical/DrillStoreRel.java     |   4 +-
 .../drill/exec/planner/logical/DrillTable.java  |  10 +-
 .../exec/planner/logical/DrillUnionRel.java     |   6 +-
 .../exec/planner/logical/DrillUnionRule.java    |   4 +-
 .../exec/planner/logical/DynamicDrillTable.java |   6 +-
 .../planner/logical/EnumerableDrillRule.java    |   4 +-
 .../planner/logical/RelDataTypeDrillImpl.java   |  78 -------
 .../exec/planner/logical/RelDataTypeHolder.java |  85 --------
 .../physical/DrillDistributionTrait.java        | 142 ++++++++++++
 .../physical/DrillDistributionTraitDef.java     |  88 ++++++++
 .../drill/exec/planner/physical/FilterPrel.java |  58 +++++
 .../exec/planner/physical/FilterPrule.java      |  58 +++++
 .../drill/exec/planner/physical/HashPrel.java   |  37 ----
 .../physical/HashToRandomExchangePrel.java      |  77 +++++++
 .../drill/exec/planner/physical/LimitPrel.java  |  95 ++++++++
 .../drill/exec/planner/physical/LimitPrule.java |  50 +++++
 .../exec/planner/physical/MergeJoinPrel.java    | 119 ++++++++++
 .../exec/planner/physical/MergeJoinPrule.java   | 101 +++++++++
 .../physical/OrderedPartitionExchangePrel.java  |  53 +++++
 .../planner/physical/PhysicalPlanCreator.java   |  79 ++++++-
 .../drill/exec/planner/physical/Prel.java       |  25 ++-
 .../drill/exec/planner/physical/PrelUtil.java   |  72 ++++++
 .../exec/planner/physical/ProjectPrel.java      |  44 +++-
 .../exec/planner/physical/ProjectPrule.java     |  26 ++-
 .../drill/exec/planner/physical/ScanPrel.java   |  39 +++-
 .../drill/exec/planner/physical/ScanPrule.java  |  32 ++-
 .../drill/exec/planner/physical/ScreenPrel.java |  46 +++-
 .../exec/planner/physical/ScreenPrule.java      |  49 +++++
 .../physical/SingleMergeExchangePrel.java       |  80 +++++++
 .../exec/planner/physical/SortConvertPrule.java |  57 +++++
 .../drill/exec/planner/physical/SortPrel.java   |  78 +++++--
 .../drill/exec/planner/physical/SortPrule.java  |  83 +++++++
 .../exec/planner/physical/StorageRule.java      |   5 -
 .../exec/planner/physical/StreamAggPrel.java    |  83 ++++++-
 .../exec/planner/physical/StreamAggPrule.java   |  94 ++++++++
 .../exec/planner/physical/TopNSortPrel.java     |  36 ---
 .../planner/physical/UnionExchangePrel.java     |  69 ++++++
 .../exec/planner/sql/DrillOperatorTable.java    |  80 +++++++
 .../exec/planner/sql/DrillSqlOperator.java      | 155 +++++++++++++
 .../drill/exec/planner/sql/DrillSqlWorker.java  | 101 +++++++--
 .../exec/planner/torel/ConversionContext.java   |  16 +-
 .../planner/types/RelDataTypeDrillImpl.java     |  86 ++++++++
 .../exec/planner/types/RelDataTypeHolder.java   |  86 ++++++++
 .../org/apache/drill/exec/schema/Field.java     |  12 +-
 .../dfs/shim/fallback/FallbackFileSystem.java   |   2 +-
 .../exec/store/easy/json/JSONRecordReader.java  |   1 +
 .../store/ischema/InfoSchemaDrillTable.java     |   4 +-
 .../store/ischema/InfoSchemaStoragePlugin.java  |  17 +-
 .../exec/store/parquet/ParquetGroupScan.java    |   7 +-
 .../apache/drill/exec/work/foreman/Foreman.java |  10 +
 .../drill/exec/work/foreman/QueryManager.java   |   2 +-
 .../org/apache/drill/TestExampleQueries.java    |  21 +-
 .../java/org/apache/drill/TestTpchQueries.java  |   2 +
 .../exec/physical/impl/join/TestMergeJoin.java  |  31 ++-
 .../planner/physical/TestPhysicalPlanning.java  |  39 ----
 .../exec/store/json/JSONRecordReaderTest.java   |   2 +-
 .../store/parquet/ParquetRecordReaderTest.java  |   3 +-
 .../resources/join/merge_join_empty_batch.json  |  47 ++++
 pom.xml                                         |   2 +-
 sample-data/nationsMF/nation.parquet            | Bin 0 -> 3109 bytes
 sample-data/nationsMF/nation2.parquet           | Bin 0 -> 3109 bytes
 sample-data/nationsSF/nation.parquet            | Bin 0 -> 3109 bytes
 sample-data/regionsMF/region.parquet            | Bin 0 -> 819 bytes
 sample-data/regionsMF/region2.parquet           | Bin 0 -> 819 bytes
 sample-data/regionsSF/region.parquet            | Bin 0 -> 819 bytes
 .../org/apache/drill/jdbc/ConnectionConfig.java |  41 ----
 .../org/apache/drill/jdbc/DrillConnection.java  |   2 +-
 .../drill/jdbc/DrillConnectionConfig.java       |  45 ++++
 .../apache/drill/jdbc/DrillConnectionImpl.java  |   6 +-
 .../drill/jdbc/test/TestJdbcDistQuery.java      | 218 +++++++++++++++++++
 .../apache/drill/jdbc/test/TestJdbcQuery.java   |   9 +-
 127 files changed, 3372 insertions(+), 1011 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
----------------------------------------------------------------------
diff --git a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
index 05fb32e..78c356a 100644
--- a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
@@ -40,19 +40,20 @@ Nullable: 'nullable';
 Repeat: 'repeat';
 As: 'as';
 
-INT		 : 'int';
-BIGINT	 : 'bigint';
-FLOAT4   : 'float4';
-FLOAT8   : 'float8';
-VARCHAR  : 'varchar';
-VARBINARY: 'varbinary';
-DATE     : 'date';
-TIMESTAMP: 'timestamp';
-TIME     : 'time';
-TIMESTAMPTZ: 'timestamptz';
-INTERVAL : 'interval';
-INTERVALYEAR : 'intervalyear';
-INTERVALDAY : 'intervalday';
+INT    : 'int' | 'INT';
+BIGINT   : 'bigint' | 'BIGINT';
+FLOAT4   : 'float4' | 'FLOAT4';
+FLOAT8   : 'float8' | 'FLOAT8';
+VARCHAR  : 'varchar' | 'VARCHAR';
+VARBINARY: 'varbinary' | 'VARBINARY';
+DATE     : 'date' | 'DATE';
+TIMESTAMP: 'timestamp' | 'TIMESTAMP';
+TIME     : 'time' | 'TIME';
+TIMESTAMPTZ: 'timestamptz' | 'TIMESTAMPTZ';
+INTERVAL : 'interval' | 'INTERVAL';
+INTERVALYEAR : 'intervalyear' | 'INTERVALYEAR';
+INTERVALDAY : 'intervalday' | 'INTERVALDAY';
+
 
 Or       : '||' | 'or' | 'OR' | 'Or';
 And      : '&&' | 'and' | 'AND' ;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/common/src/main/java/org/apache/drill/common/logical/data/Order.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Order.java b/common/src/main/java/org/apache/drill/common/logical/data/Order.java
index 3c864b0..06f1144 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/Order.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Order.java
@@ -100,7 +100,7 @@ public class Order extends SingleInputOperator {
     public String getOrder() {
       
       switch(direction){
-      case Descending: return "DESC";
+      case DESCENDING: return "DESC";
       default: return "ASC";
       }
     }
@@ -140,10 +140,10 @@ public class Order extends SingleInputOperator {
   }
   
   public static Direction getDirectionFromString(String direction){
-    return "DESC".equalsIgnoreCase(direction) ? Direction.Descending : Direction.Ascending;
+    return "DESC".equalsIgnoreCase(direction) ? Direction.DESCENDING : Direction.ASCENDING;
   }
   
   public static String getStringFromDirection(Direction direction){
-    return direction == Direction.Descending ? "DESC" : "ASC";
+    return direction == Direction.DESCENDING ? "DESC" : "ASC";
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
index 67191ec..8c05124 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseGroupScan.java
@@ -25,7 +25,6 @@ import com.google.common.collect.Iterators;
 import com.google.common.collect.Lists;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
-import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.exec.physical.EndpointAffinity;
 import org.apache.drill.exec.physical.OperatorCost;
@@ -70,7 +69,6 @@ public class HBaseGroupScan extends AbstractGroupScan {
   private String tableName;
   private HBaseStoragePlugin storagePlugin;
   private HBaseStoragePluginConfig storagePluginConfig;
-  private final FieldReference ref;
   private List<EndpointAffinity> endpointAffinities;
   private List<SchemaPath> columns;
 
@@ -80,24 +78,21 @@ public class HBaseGroupScan extends AbstractGroupScan {
   public HBaseGroupScan(@JsonProperty("entries") List<HTableReadEntry> entries,
                           @JsonProperty("storage") HBaseStoragePluginConfig storageEngineConfig,
                           @JsonProperty("columns") List<SchemaPath> columns,
-                          @JacksonInject StoragePluginRegistry engineRegistry,
-                          @JsonProperty("ref") FieldReference ref
+                          @JacksonInject StoragePluginRegistry engineRegistry
                            )throws IOException, ExecutionSetupException {
     Preconditions.checkArgument(entries.size() == 1);
     engineRegistry.init(DrillConfig.create());
     this.storagePlugin = (HBaseStoragePlugin) engineRegistry.getEngine(storageEngineConfig);
     this.storagePluginConfig = storageEngineConfig;
     this.tableName = entries.get(0).getTableName();
-    this.ref = ref;
     this.columns = columns;
     getRegionInfos();
   }
 
-  public HBaseGroupScan(String tableName, HBaseStoragePlugin storageEngine, FieldReference ref, List<SchemaPath> columns) throws IOException {
+  public HBaseGroupScan(String tableName, HBaseStoragePlugin storageEngine, List<SchemaPath> columns) throws IOException {
     this.storagePlugin = storageEngine;
-    this.storagePluginConfig = storageEngine.getEngineConfig();
+    this.storagePluginConfig = storageEngine.getConfig();
     this.tableName = tableName;
-    this.ref = ref;
     this.columns = columns;
     getRegionInfos();
   }
@@ -167,12 +162,9 @@ public class HBaseGroupScan extends AbstractGroupScan {
 
   @Override
   public HBaseSubScan getSpecificScan(int minorFragmentId) {
-    return new HBaseSubScan(storagePlugin, storagePluginConfig, mappings.get(minorFragmentId), ref, columns);
+    return new HBaseSubScan(storagePlugin, storagePluginConfig, mappings.get(minorFragmentId), columns);
   }
 
-  public FieldReference getRef() {
-    return ref;
-  }
 
   @Override
   public int getMaxParallelizationWidth() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index 895433f..d97f3d0 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -35,11 +35,11 @@ import org.apache.hadoop.hbase.client.HBaseAdmin;
 public class HBaseSchemaFactory implements SchemaFactory {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseSchemaFactory.class);
 
-  final HBaseStoragePluginConfig configuration;
   final String schemaName;
-
-  public HBaseSchemaFactory(HBaseStoragePluginConfig configuration, String name) throws IOException {
-    this.configuration = configuration;
+  final HBaseStoragePlugin plugin;
+  
+  public HBaseSchemaFactory(HBaseStoragePlugin plugin, String name) throws IOException {
+    this.plugin = plugin;
     this.schemaName = name;
   }
 
@@ -65,7 +65,7 @@ public class HBaseSchemaFactory implements SchemaFactory {
 
     @Override
     public Schema getSubSchema(String name) {
-      throw new UnsupportedOperationException();
+      return null;
     }
 
     @Override
@@ -76,13 +76,12 @@ public class HBaseSchemaFactory implements SchemaFactory {
     @Override
     public DrillTable getTable(String name) {
       Object selection = new HTableReadEntry(name);
-      return new DynamicDrillTable(schemaName, selection, configuration);
+      return new DynamicDrillTable(plugin, schemaName, selection, plugin.getConfig());
     }
 
     @Override
     public Set<String> getTableNames() {
-      try {
-        HBaseAdmin admin = new HBaseAdmin(configuration.conf);
+      try(HBaseAdmin admin = new HBaseAdmin(plugin.getConfig().conf)) {
         HTableDescriptor[] tables = admin.listTables();
         Set<String> tableNames = Sets.newHashSet();
         for (HTableDescriptor table : tables) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
index 3bbae2d..1fab650 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
@@ -18,18 +18,17 @@
 package org.apache.drill.exec.store.hbase;
 
 import java.io.IOException;
-import java.util.ArrayList;
-
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
 
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
-import org.apache.drill.common.logical.data.Scan;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.common.logical.StoragePluginConfig;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 
-import com.google.common.base.Preconditions;
+import com.fasterxml.jackson.core.type.TypeReference;
+import com.fasterxml.jackson.databind.ObjectMapper;
 
 public class HBaseStoragePlugin extends AbstractStoragePlugin {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseStoragePlugin.class);
@@ -42,15 +41,11 @@ public class HBaseStoragePlugin extends AbstractStoragePlugin {
   public HBaseStoragePlugin(HBaseStoragePluginConfig configuration, DrillbitContext context, String name)
       throws IOException {
     this.context = context;
-    this.schemaFactory = new HBaseSchemaFactory(configuration, name);
+    this.schemaFactory = new HBaseSchemaFactory(this, name);
     this.engineConfig = configuration;
     this.name = name;
   }
 
-  public HBaseStoragePluginConfig getEngineConfig() {
-    return engineConfig;
-  }
-
   public DrillbitContext getContext() {
     return this.context;
   }
@@ -61,11 +56,10 @@ public class HBaseStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public HBaseGroupScan getPhysicalScan(Scan scan) throws IOException {
-    HTableReadEntry readEntry = scan.getSelection().getListWith(new ObjectMapper(),
+  public HBaseGroupScan getPhysicalScan(JSONOptions selection) throws IOException {
+    HTableReadEntry readEntry = selection.getListWith(new ObjectMapper(),
         new TypeReference<HTableReadEntry>() {});
-
-    return new HBaseGroupScan(readEntry.getTableName(), this, scan.getOutputReference(), null);
+    return new HBaseGroupScan(readEntry.getTableName(), this, null);
   }
 
   @Override
@@ -73,4 +67,9 @@ public class HBaseStoragePlugin extends AbstractStoragePlugin {
     return schemaFactory.add(parent);
   }
 
+  @Override
+  public HBaseStoragePluginConfig getConfig() {
+    return engineConfig;
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
index 8c4eea0..0e8a934 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSubScan.java
@@ -51,28 +51,24 @@ public class HBaseSubScan extends AbstractBase implements SubScan {
   @JsonIgnore
   private final HBaseStoragePlugin hbaseStoragePlugin;
   private final List<HBaseSubScanReadEntry> rowGroupReadEntries;
-  private final FieldReference ref;
   private final List<SchemaPath> columns;
 
   @JsonCreator
   public HBaseSubScan(@JacksonInject StoragePluginRegistry registry, @JsonProperty("storage") StoragePluginConfig storage,
                       @JsonProperty("rowGroupReadEntries") LinkedList<HBaseSubScanReadEntry> rowGroupReadEntries,
-                      @JsonProperty("columns") List<SchemaPath> columns,
-                      @JsonProperty("ref") FieldReference ref) throws ExecutionSetupException {
+                      @JsonProperty("columns") List<SchemaPath> columns) throws ExecutionSetupException {
     hbaseStoragePlugin = (HBaseStoragePlugin) registry.getEngine(storage);
     this.rowGroupReadEntries = rowGroupReadEntries;
     this.storage = storage;
-    this.ref = ref;
     this.columns = columns;
   }
 
   public HBaseSubScan(HBaseStoragePlugin plugin, HBaseStoragePluginConfig config,
-                      List<HBaseSubScanReadEntry> regionInfoList, FieldReference ref,
+                      List<HBaseSubScanReadEntry> regionInfoList,
                       List<SchemaPath> columns) {
     hbaseStoragePlugin = plugin;
     storage = config;
     this.rowGroupReadEntries = regionInfoList;
-    this.ref = ref;
     this.columns = columns;
   }
 
@@ -94,11 +90,6 @@ public class HBaseSubScan extends AbstractBase implements SubScan {
     return null;
   }
 
-  
-  public FieldReference getRef() {
-    return ref;
-  }
-
   @Override
   public Size getSize() {
     return null;
@@ -122,7 +113,7 @@ public class HBaseSubScan extends AbstractBase implements SubScan {
   @Override
   public PhysicalOperator getNewWithChildren(List<PhysicalOperator> children) {
     Preconditions.checkArgument(children.isEmpty());
-    return new HBaseSubScan(hbaseStoragePlugin, (HBaseStoragePluginConfig) storage, rowGroupReadEntries, ref, columns);
+    return new HBaseSubScan(hbaseStoragePlugin, (HBaseStoragePluginConfig) storage, rowGroupReadEntries, columns);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashFunctions.java
index 5a6943e..36300ee 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/HashFunctions.java
@@ -106,6 +106,23 @@ public class HashFunctions {
     }
   }
 
+  @FunctionTemplate(name = "hash", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL )
+  public static class NullableVarCharHash implements DrillSimpleFunc {
+
+    @Param NullableVarCharHolder in;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch incoming) {
+    }
+
+    public void eval() {
+      if (in.isSet == 0)
+        out.value = 0;
+      else
+        out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash(in.buffer.nioBuffer(), 0);
+    }
+  }
+
   @FunctionTemplate(name = "hash", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class NullableBigIntHash implements DrillSimpleFunc {
 
@@ -154,6 +171,20 @@ public class HashFunctions {
       out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash(in.buffer.nioBuffer(in.start, in.end - in.start), 0);
     }
   }
+
+  @FunctionTemplate(name = "hash", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class VarCharHash implements DrillSimpleFunc {
+
+    @Param VarCharHolder in;
+    @Output IntHolder out;
+
+    public void setup(RecordBatch incoming) {
+    }
+
+    public void eval() {
+      out.value = org.apache.drill.exec.expr.fn.impl.HashHelper.hash(in.buffer.nioBuffer(in.start, in.end - in.start), 0);
+    }
+  }
   
   @FunctionTemplate(name = "hash", scope = FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
   public static class HashBigInt implements DrillSimpleFunc {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index fa66926..1564b09 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -21,6 +21,7 @@ import java.util.Collection;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.cache.DistributedCache;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.planner.PhysicalPlanReader;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
@@ -85,4 +86,7 @@ public class QueryContext {
     return drillbitContext.getSchemaFactory();
   }
   
+  public FunctionImplementationRegistry getFunctionRegistry(){
+    return drillbitContext.getFunctionImplementationRegistry();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 36b8373..97ec026 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -129,7 +129,7 @@ public class BasicOptimizer extends Optimizer{
 
       if(groupBy.getKeys().length > 0){
         for(NamedExpression e : groupBy.getKeys()){
-          orderDefs.add(new Ordering(Direction.Ascending, e.getExpr(), NullDirection.FIRST));
+          orderDefs.add(new Ordering(Direction.ASCENDING, e.getExpr(), NullDirection.FIRST));
         }
         input = new Sort(input, orderDefs, false);
       }
@@ -161,7 +161,7 @@ public class BasicOptimizer extends Optimizer{
       PhysicalOperator leftOp = join.getLeft().accept(this, value);
       List<Ordering> leftOrderDefs = Lists.newArrayList();
       for(JoinCondition jc : join.getConditions()){
-        leftOrderDefs.add(new Ordering(Direction.Ascending, jc.getLeft()));
+        leftOrderDefs.add(new Ordering(Direction.ASCENDING, jc.getLeft()));
       }
       leftOp = new Sort(leftOp, leftOrderDefs, false);
       leftOp = new SelectionVectorRemover(leftOp);
@@ -169,7 +169,7 @@ public class BasicOptimizer extends Optimizer{
       PhysicalOperator rightOp = join.getRight().accept(this, value);
       List<Ordering> rightOrderDefs = Lists.newArrayList();
       for(JoinCondition jc : join.getConditions()){
-        rightOrderDefs.add(new Ordering(Direction.Ascending, jc.getRight()));
+        rightOrderDefs.add(new Ordering(Direction.ASCENDING, jc.getRight()));
       }
       rightOp = new Sort(rightOp, rightOrderDefs, false);
       rightOp = new SelectionVectorRemover(rightOp);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
index f7a6f51..7eced4d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractBase.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.physical.base;
 
 import org.apache.drill.common.graph.GraphVisitor;
 import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.google.common.base.Preconditions;
 
@@ -43,4 +44,9 @@ public abstract class AbstractBase implements PhysicalOperator{
     return true;
   }
   
+  @Override
+  public SelectionVectorMode getSVMode() {
+    return SelectionVectorMode.NONE;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSubScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSubScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSubScan.java
index c6e11d1..57b9c18 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSubScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/AbstractSubScan.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.graph.GraphVisitor;
 import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.google.common.collect.Iterators;
 
@@ -67,4 +68,9 @@ public abstract class AbstractSubScan implements SubScan{
     return Iterators.emptyIterator();
   }
   
+  @Override
+  public SelectionVectorMode getSVMode() {
+    return SelectionVectorMode.NONE;
+  }
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index 314bc84..314e889 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -21,7 +21,6 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
-import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -30,7 +29,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
  * A GroupScan operator represents all data which will be scanned by a given physical
  * plan.  It is the superset of all SubScans for the plan.
  */
-public interface GroupScan extends Scan, HasAffinity, Prel{
+public interface GroupScan extends Scan, HasAffinity{ 
 
   public abstract void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
index 6872595..66e1b46 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/PhysicalOperator.java
@@ -22,6 +22,7 @@ import java.util.List;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.graph.GraphValue;
 import org.apache.drill.exec.physical.OperatorCost;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonIdentityInfo;
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -63,6 +64,14 @@ public interface  PhysicalOperator extends GraphValue<PhysicalOperator> {
   public boolean isExecutable();
 
   /**
+   * Describes the SelectionVector Mode for the output steam from this physical op.
+   * This property is used during physical plan creating using {@link PhysicalPlanCreator}.
+   * @return
+   */
+  @JsonIgnore
+  public SelectionVectorMode getSVMode();
+  
+  /**
    * Provides capability to build a set of output based on traversing a query graph tree.
    * 
    * @param physicalVisitor

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
index aba1747..f125d05 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Filter.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -66,10 +67,14 @@ public class Filter extends AbstractSingle {
   public Size getSize() {
     return new Size( (long) (child.getSize().getRecordCount()*selectivity), child.getSize().getRecordSize());
   }
-  
-  
-
-  
-  
+   
+  @Override
+  public SelectionVectorMode getSVMode() {
+    if (child.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
+      return SelectionVectorMode.FOUR_BYTE;
+    } else {
+      return SelectionVectorMode.TWO_BYTE;
+    }
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
index 9d5f3d6..61c2f1a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/HashToRandomExchange.java
@@ -84,7 +84,10 @@ public class HashToRandomExchange extends AbstractExchange{
     return true;
   }
 
-  
+  @JsonProperty("expr")
+  public LogicalExpression getExpression(){
+    return expr;
+  }
 
   
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
index fde88a9..a9a0f6d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
@@ -28,7 +28,6 @@ import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
 import org.eigenbase.rel.JoinRelType;
-import org.eigenbase.sql.SqlJoinOperator.JoinType;
 
 import com.beust.jcommander.internal.Lists;
 import com.fasterxml.jackson.annotation.JsonCreator;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
index 7ab0620..0e6b0fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Project.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -67,5 +68,8 @@ public class Project extends AbstractSingle{
     return new Project(exprs, child);
   }
   
-  
+  @Override
+  public SelectionVectorMode getSVMode() {
+    return child.getSVMode();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
index 654d1f7..c4d78f9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Screen.java
@@ -45,6 +45,7 @@ public class Screen extends AbstractStore {
 
   public Screen(@JsonProperty("child") PhysicalOperator child, @JacksonInject DrillbitEndpoint endpoint) {
     super(child);
+    assert (endpoint!=null);
     this.endpoint = endpoint;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleMergeExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleMergeExchange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleMergeExchange.java
index b8073c1..df13a84 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleMergeExchange.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SingleMergeExchange.java
@@ -89,4 +89,9 @@ public class SingleMergeExchange extends AbstractExchange {
     return true;
   }
 
+  @JsonProperty("orderings")
+  public List<Ordering> getOrderings() {
+    return this.orderExpr;
+  }
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
index 507a960..c9ac137 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/Sort.java
@@ -25,6 +25,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -79,7 +80,10 @@ public class Sort extends AbstractSingle{
     return new Sort(child, orderings, reverse);
   }
 
-    
+  @Override
+  public SelectionVectorMode getSVMode() {
+    return SelectionVectorMode.FOUR_BYTE;
+  }
   
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
index 3a53555..138c1cf 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/TopNBatch.java
@@ -250,7 +250,7 @@ public class TopNBatch extends AbstractRecordBatch<TopN> {
       HoldingContainer out = g.addExpr(fh, false);
       JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
       
-      if(od.getDirection() == Direction.Ascending){
+      if(od.getDirection() == Direction.ASCENDING){
         jc._then()._return(out.getValue());
       }else{
         jc._then()._return(out.getValue().minus());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
index fb91b2e..5a83b48 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
@@ -206,12 +206,12 @@ public final class JoinStatus {
       return JoinOutcome.FAILURE;
     if (lastLeft == IterOutcome.OK && lastRight == IterOutcome.OK)
       return JoinOutcome.BATCH_RETURNED;
+    if (eitherMatches(IterOutcome.NONE))
+      return JoinOutcome.NO_MORE_DATA;    
     if (eitherMatches(IterOutcome.OK_NEW_SCHEMA))
       return JoinOutcome.SCHEMA_CHANGED;
     if (eitherMatches(IterOutcome.NOT_YET))
       return JoinOutcome.WAITING;
-    if (eitherMatches(IterOutcome.NONE))
-      return JoinOutcome.NO_MORE_DATA;
     return JoinOutcome.FAILURE;
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index db5bccf..0cb2f7d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -128,17 +128,25 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     // loop so we can start over again if we find a new batch was created.
     while(true){
 
+      JoinOutcome outcome = status.getOutcome();
       // if the previous outcome was a change in schema or we sent a batch, we have to set up a new batch.
-      if (status.getOutcome() == JoinOutcome.BATCH_RETURNED ||
-          status.getOutcome() == JoinOutcome.SCHEMA_CHANGED)
+      if (outcome == JoinOutcome.BATCH_RETURNED ||
+          outcome == JoinOutcome.SCHEMA_CHANGED)
         allocateBatch();
 
       // reset the output position to zero after our parent iterates this RecordBatch
-      if (status.getOutcome() == JoinOutcome.BATCH_RETURNED ||
-          status.getOutcome() == JoinOutcome.SCHEMA_CHANGED ||
-          status.getOutcome() == JoinOutcome.NO_MORE_DATA)
+      if (outcome == JoinOutcome.BATCH_RETURNED ||
+          outcome == JoinOutcome.SCHEMA_CHANGED ||
+          outcome == JoinOutcome.NO_MORE_DATA)
         status.resetOutputPos();
 
+      if (outcome == JoinOutcome.NO_MORE_DATA) {
+        left.cleanup();
+        right.cleanup();
+        logger.debug("NO MORE DATA; returning {}  NONE");
+        return IterOutcome.NONE;
+      }
+      
       boolean first = false;
       if(worker == null){
         try {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
index bf2bcf8..dcd452e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/limit/LimitRecordBatch.java
@@ -80,6 +80,7 @@ public class LimitRecordBatch extends AbstractSingleRecordBatch<Limit> {
   public IterOutcome next() {
     if(!noEndLimit && recordsLeft <= 0) {
       killIncoming();
+      cleanup();
       return IterOutcome.NONE;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index d6d42b6..13ed4c9 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -546,11 +546,11 @@ public class MergingRecordBatch implements RecordBatch {
       // generate less than/greater than checks (fixing results for ASCending vs. DESCending)
       cg.getEvalBlock()._if(out.getValue().eq(JExpr.lit(1)))
                        ._then()
-                       ._return(JExpr.lit(config.getOrderings().get(comparisonVectorIndex).getDirection() == Direction.Ascending ? 1 : -1));
+                       ._return(JExpr.lit(config.getOrderings().get(comparisonVectorIndex).getDirection() == Direction.ASCENDING ? 1 : -1));
 
       cg.getEvalBlock()._if(out.getValue().eq(JExpr.lit(-1)))
                        ._then()
-                       ._return(JExpr.lit(config.getOrderings().get(comparisonVectorIndex).getDirection() == Direction.Ascending ? -1 : 1));
+                       ._return(JExpr.lit(config.getOrderings().get(comparisonVectorIndex).getDirection() == Direction.ASCENDING ? -1 : 1));
 
       ++comparisonVectorIndex;
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index 4eee6af..5fc3733 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -533,7 +533,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
       ClassGenerator.HoldingContainer out = cg.addExpr(fh, false);
       JConditional jc = cg.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
 
-      if (od.getDirection() == Direction.Ascending) {
+      if (od.getDirection() == Direction.ASCENDING) {
         jc._then()._return(out.getValue());
       } else {
         jc._then()._return(out.getValue().minus());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
index 51ba418..5cdab96 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
@@ -189,7 +189,7 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
       HoldingContainer out = g.addExpr(fh, false);
       JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
       
-      if(od.getDirection() == Direction.Ascending){
+      if(od.getDirection() == Direction.ASCENDING){
         jc._then()._return(out.getValue());
       }else{
         jc._then()._return(out.getValue().minus());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 4dc47b6..0a3b768 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -389,7 +389,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       HoldingContainer out = g.addExpr(fh, false);
       JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
 
-      if(od.getDirection() == Direction.Ascending){
+      if(od.getDirection() == Direction.ASCENDING){
         jc._then()._return(out.getValue());
       }else{
         jc._then()._return(out.getValue().minus());
@@ -432,7 +432,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       HoldingContainer out = g.addExpr(fh, false);
       JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
 
-      if(od.getDirection() == Direction.Ascending){
+      if(od.getDirection() == Direction.ASCENDING){
         jc._then()._return(out.getValue());
       }else{
         jc._then()._return(out.getValue().minus());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
deleted file mode 100644
index 42c0ec5..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
+++ /dev/null
@@ -1,53 +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.planner.common;
-
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.logical.data.Filter;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillOptiq;
-import org.apache.drill.exec.planner.logical.DrillParseContext;
-import org.apache.drill.exec.planner.logical.DrillRel;
-import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.eigenbase.rel.FilterRelBase;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexNode;
-
-
-public abstract class BaseFilterRel extends FilterRelBase{
-  protected BaseFilterRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
-    super(cluster, traits, child, condition);
-    assert getConvention() == convention;
-  }
-  
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
-  }
-
-  protected LogicalExpression getFilterExpression(DrillParseContext context){
-    return DrillOptiq.toDrill(context, getChild(), getCondition());
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
deleted file mode 100644
index 9314a29..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
+++ /dev/null
@@ -1,68 +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.planner.common;
-
-import java.util.List;
-
-import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.LogicalExpression;
-import org.apache.drill.common.logical.data.NamedExpression;
-import org.apache.drill.exec.planner.logical.DrillOptiq;
-import org.apache.drill.exec.planner.logical.DrillParseContext;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.util.Pair;
-
-import com.google.common.collect.Lists;
-
-/**
- * Project implemented in Drill.
- */
-public class BaseProjectRel extends ProjectRelBase{
-  protected BaseProjectRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
-      RelDataType rowType) {
-    super(cluster, traits, child, exps, rowType, Flags.BOXED);
-    assert getConvention() == convention;
-  }
-
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
-  }
-
-  private List<Pair<RexNode, String>> projects() {
-    return Pair.zip(exps, getRowType().getFieldNames());
-  }
-
-  protected List<NamedExpression> getProjectExpressions(DrillParseContext context){
-    List<NamedExpression> expressions = Lists.newArrayList();
-    for (Pair<RexNode, String> pair : projects()) {
-      LogicalExpression expr = DrillOptiq.toDrill(context, getChild(), pair.left);
-      expressions.add(new NamedExpression(expr, new FieldReference("output." + pair.right)));
-    }
-    return expressions;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
deleted file mode 100644
index 69b47ab..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
+++ /dev/null
@@ -1,36 +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.planner.common;
-
-import org.apache.drill.exec.planner.logical.DrillTable;
-import org.eigenbase.rel.TableAccessRelBase;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptTable;
-import org.eigenbase.relopt.RelTraitSet;
-
-public abstract class BaseScanRel extends TableAccessRelBase{
-  protected final DrillTable drillTable;
-
-  public BaseScanRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
-    super(cluster, traits, table);
-    this.drillTable = table.unwrap(DrillTable.class);
-    assert drillTable != null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
deleted file mode 100644
index 4c1a318..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
+++ /dev/null
@@ -1,41 +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.planner.common;
-
-import org.apache.drill.exec.planner.logical.DrillRel;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SingleRel;
-import org.eigenbase.relopt.Convention;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
-import org.eigenbase.relopt.RelTraitSet;
-
-public abstract class BaseScreenRel extends SingleRel {
-
-  public BaseScreenRel(Convention convention, RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
-    super(cluster, traitSet, input);
-    assert input.getConvention() == convention;
-  }
-
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.java
new file mode 100644
index 0000000..4854307
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillAggregateRelBase.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.planner.common;
+
+import java.util.BitSet;
+import java.util.List;
+
+
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+
+/**
+ * Base class for logical and physical Aggregations implemented in Drill
+ */
+public abstract class DrillAggregateRelBase extends AggregateRelBase implements DrillRelNode {
+
+  public DrillAggregateRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
+      List<AggregateCall> aggCalls) throws InvalidRelException {
+    super(cluster, traits, child, groupSet, aggCalls);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.java
new file mode 100644
index 0000000..955729b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillFilterRelBase.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.planner.common;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Filter;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.FilterRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
+
+/**
+ * Base class for logical and physical Filters implemented in Drill
+ */
+public abstract class DrillFilterRelBase extends FilterRelBase implements DrillRelNode {
+  protected DrillFilterRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+    super(cluster, traits, child, condition);
+    assert getConvention() == convention;
+  }
+  
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+  }
+
+  protected LogicalExpression getFilterExpression(DrillParseContext context){
+    return DrillOptiq.toDrill(context, getChild(), getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.java
new file mode 100644
index 0000000..e88faf9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillJoinRelBase.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.planner.common;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.JoinRelBase;
+import org.eigenbase.rel.JoinRelType;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.util.Pair;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Base class for logical and physical Joins implemented in Drill.
+ */
+public abstract class DrillJoinRelBase extends JoinRelBase implements DrillRelNode {
+  protected final List<Integer> leftKeys = new ArrayList<>();
+  protected final List<Integer> rightKeys = new ArrayList<>();
+
+  public DrillJoinRelBase(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
+      JoinRelType joinType) throws InvalidRelException {
+    super(cluster, traits, left, right, condition, joinType, Collections.<String> emptySet());
+  }
+  
+  
+  /**
+   * Returns whether there are any elements in common between left and right.
+   */
+  private static <T> boolean intersects(List<T> left, List<T> right) {
+    return new HashSet<>(left).removeAll(right);
+  }
+
+  protected boolean uniqueFieldNames(RelDataType rowType) {
+    return isUnique(rowType.getFieldNames());
+  }
+
+  protected static <T> boolean isUnique(List<T> list) {
+    return new HashSet<>(list).size() == list.size();
+  }
+  
+  public List<Integer> getLeftKeys() {
+    return this.leftKeys;
+  }
+  
+  public List<Integer> getRightKeys() {
+    return this.rightKeys;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
deleted file mode 100644
index cfaf350..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
+++ /dev/null
@@ -1,75 +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.planner.common;
-
-import java.math.BigDecimal;
-import java.util.List;
-
-import org.apache.drill.common.logical.data.Limit;
-import org.apache.drill.common.logical.data.LogicalOperator;
-import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillRel;
-import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SingleRel;
-import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.rex.RexLiteral;
-import org.eigenbase.rex.RexNode;
-import org.eigenbase.sql.type.SqlTypeName;
-
-public class DrillLimitRel extends SingleRel implements DrillRel {
-  private RexNode offset;
-  private RexNode fetch;
-
-  public DrillLimitRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
-    super(cluster, traitSet, child);
-    this.offset = offset;
-    this.fetch = fetch;
-  }
-
-  @Override
-  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
-    return new DrillLimitRel(getCluster(), traitSet, sole(inputs), offset, fetch);
-  }
-
-  @Override
-  public LogicalOperator implement(DrillImplementor implementor) {
-    LogicalOperator inputOp = implementor.visitChild(this, 0, getChild());
-    
-    // First offset to include into results (inclusive). Null implies it is starting from offset 0
-    int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
-
-    // Last offset to stop including into results (exclusive), translating fetch row counts into an offset.
-    // Null value implies including entire remaining result set from first offset
-    Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
-    Limit limit = new Limit(first, last);
-    limit.setInput(inputOp);
-    return limit;
-  }
-  
-  public static DrillLimitRel convert(Limit limit, ConversionContext context) throws InvalidRelException{
-    RelNode input = context.toRel(limit.getInput());
-    RexNode first = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getFirst()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
-    RexNode last = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getLast()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
-    return new DrillLimitRel(context.getCluster(), context.getLogicalTraits(), input, first, last);
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
new file mode 100644
index 0000000..c64b79b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
@@ -0,0 +1,57 @@
+/**
+ * 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.common;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.Limit;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.type.SqlTypeName;
+
+/**
+ * Base class for logical and physical Limits implemented in Drill
+ */
+public abstract class DrillLimitRelBase extends SingleRel implements DrillRelNode {
+  protected RexNode offset;
+  protected RexNode fetch;
+
+  public DrillLimitRelBase(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
+    super(cluster, traitSet, child);
+    this.offset = offset;
+    this.fetch = fetch;
+  }
+
+  public RexNode getOffset() {
+    return this.offset;
+  }
+  
+  public RexNode getFetch() {
+    return this.fetch;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
new file mode 100644
index 0000000..9d02c44
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.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.planner.common;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.eigenbase.rel.ProjectRelBase;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.util.Pair;
+
+import com.google.common.collect.Lists;
+
+/**
+ *
+ * Base class for logical and physical Project implemented in Drill
+ */ 
+public abstract class DrillProjectRelBase extends ProjectRelBase implements DrillRelNode {
+  protected DrillProjectRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
+      RelDataType rowType) {
+    super(cluster, traits, child, exps, rowType, Flags.BOXED);
+    assert getConvention() == convention;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+  }
+
+  private List<Pair<RexNode, String>> projects() {
+    return Pair.zip(exps, getRowType().getFieldNames());
+  }
+
+  protected List<NamedExpression> getProjectExpressions(DrillParseContext context) {
+    List<NamedExpression> expressions = Lists.newArrayList();
+    for (Pair<RexNode, String> pair : projects()) {
+      LogicalExpression expr = DrillOptiq.toDrill(context, getChild(), pair.left);
+      expressions.add(new NamedExpression(expr, new FieldReference("output." + pair.right)));
+    }
+    return expressions;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelNode.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelNode.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelNode.java
new file mode 100644
index 0000000..fd04cdd
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillRelNode.java
@@ -0,0 +1,25 @@
+/**
+ * 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.common;
+
+import org.eigenbase.rel.RelNode;
+
+public interface DrillRelNode extends RelNode {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
new file mode 100644
index 0000000..b370352
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScanRelBase.java
@@ -0,0 +1,39 @@
+/**
+ * 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.common;
+
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.eigenbase.rel.TableAccessRelBase;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+
+/**
+ * Base class for logical and physical Scans implemented in Drill
+ */
+public abstract class DrillScanRelBase extends TableAccessRelBase implements DrillRelNode {
+  protected final DrillTable drillTable;
+
+  public DrillScanRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
+    super(cluster, traits, table);
+    this.drillTable = table.unwrap(DrillTable.class);
+    assert drillTable != null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/6b517daa/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScreenRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScreenRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScreenRelBase.java
new file mode 100644
index 0000000..51ed442
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillScreenRelBase.java
@@ -0,0 +1,43 @@
+/**
+ * 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.common;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+/**
+ * Base class for logical and physical Screen implemented in Drill
+ */
+public abstract class DrillScreenRelBase extends SingleRel implements DrillRelNode {
+
+  public DrillScreenRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
+    super(cluster, traitSet, input);
+    assert input.getConvention() == convention;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(.1);
+  }
+
+}


[42/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
index 4dfb309..6027c44 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/foreman/Foreman.java
@@ -34,7 +34,6 @@ import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.ExecConstants;
 import org.apache.drill.exec.exception.FragmentSetupException;
 import org.apache.drill.exec.exception.OptimizerException;
-import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.opt.BasicOptimizer;
 import org.apache.drill.exec.physical.PhysicalPlan;
@@ -46,8 +45,8 @@ import org.apache.drill.exec.planner.fragment.MakeFragmentsVisitor;
 import org.apache.drill.exec.planner.fragment.PlanningSet;
 import org.apache.drill.exec.planner.fragment.SimpleParallelizer;
 import org.apache.drill.exec.planner.fragment.StatsCollector;
+import org.apache.drill.exec.planner.sql.DirectPlan;
 import org.apache.drill.exec.planner.sql.DrillSqlWorker;
-import org.apache.drill.exec.planner.sql.DrillSqlWorker.RelResult;
 import org.apache.drill.exec.proto.BitControl.PlanFragment;
 import org.apache.drill.exec.proto.GeneralRPCProtos.Ack;
 import org.apache.drill.exec.proto.UserBitShared.DrillPBError;
@@ -85,23 +84,23 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   private WorkerBee bee;
   private UserClientConnection initiatingClient;
   private final AtomicState<QueryState> state;
-  
+
   public Foreman(WorkerBee bee, DrillbitContext dContext, UserClientConnection connection, QueryId queryId,
       RunQuery queryRequest) {
     this.queryId = queryId;
     this.queryRequest = queryRequest;
-    this.context = new QueryContext(queryId, dContext);
+    this.context = new QueryContext(connection.getSession(), queryId, dContext);
     this.initiatingClient = connection;
     this.fragmentManager = new QueryManager(new ForemanManagerListener(), dContext.getController());
     this.bee = bee;
-    
+
     this.state = new AtomicState<QueryState>(QueryState.PENDING) {
       protected QueryState getStateFromNumber(int i) {
         return QueryState.valueOf(i);
       }
     };
   }
-  
+
   private boolean isFinished(){
     switch(state.getState()){
     case PENDING:
@@ -110,7 +109,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     default:
       return true;
     }
-    
+
   }
 
   private void fail(String message, Throwable t) {
@@ -128,19 +127,19 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     cleanupAndSendResult(result);
   }
 
-  
+
   public void cancel() {
     if(isFinished()){
       return;
     }
-    
+
     // cancel remote fragments.
     fragmentManager.cancel();
-    
+
     QueryResult result = QueryResult.newBuilder().setQueryState(QueryState.CANCELED).setIsLastChunk(true).setQueryId(queryId).build();
     cleanupAndSendResult(result);
   }
-  
+
   void cleanupAndSendResult(QueryResult result){
     bee.retireForeman(this);
     initiatingClient.sendResult(new ResponseSendListener(), new QueryWritableBatch(result));
@@ -154,7 +153,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
               ex);
     }
   }
-  
+
 
 
   /**
@@ -164,10 +163,10 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     // convert a run query request into action
     try{
       switch (queryRequest.getType()) {
-      
+
       case LOGICAL:
         parseAndRunLogicalPlan(queryRequest.getPlan());
-        
+
         break;
       case PHYSICAL:
         parseAndRunPhysicalPlan(queryRequest.getPlan());
@@ -178,27 +177,31 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       default:
         throw new UnsupportedOperationException();
       }
-    }catch(Exception ex){
+    }catch(AssertionError | Exception ex){
       fail("Failure while setting up Foreman.", ex);
+    }catch(OutOfMemoryError e){
+      System.out.println("Out of memory, exiting.");
+      System.out.flush();
+      System.exit(-1);
     }
   }
 
   private void parseAndRunLogicalPlan(String json) {
-    
+
     try {
       LogicalPlan logicalPlan = context.getPlanReader().readLogicalPlan(json);
-      
+
       if(logicalPlan.getProperties().resultMode == ResultMode.LOGICAL){
         fail("Failure running plan.  You requested a result mode of LOGICAL and submitted a logical plan.  In this case you're output mode must be PHYSICAL or EXEC.", new Exception());
-      }      
+      }
       if(logger.isDebugEnabled()) logger.debug("Logical {}", logicalPlan.unparse(context.getConfig()));
       PhysicalPlan physicalPlan = convert(logicalPlan);
-      
+
       if(logicalPlan.getProperties().resultMode == ResultMode.PHYSICAL){
         returnPhysical(physicalPlan);
         return;
       }
-      
+
       if(logger.isDebugEnabled()) logger.debug("Physical {}", context.getConfig().getMapper().writeValueAsString(physicalPlan));
       runPhysicalPlan(physicalPlan);
     } catch (IOException e) {
@@ -208,59 +211,31 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     }
   }
 
-  
-  private void returnLogical(LogicalPlan plan){
-    String jsonPlan = plan.toJsonStringSafe(context.getConfig());
-    sendSingleString("logical", jsonPlan);
-  }
-  
   private void returnPhysical(PhysicalPlan plan){
     String jsonPlan = plan.unparse(context.getConfig().getMapper().writer());
-    sendSingleString("physical", jsonPlan);
+    runPhysicalPlan(DirectPlan.createDirectPlan(context, new PhysicalFromLogicalExplain(jsonPlan)));
   }
-  
-  private void sendSingleString(String columnName, String value){
-    MaterializedField f = MaterializedField.create(new SchemaPath(columnName, ExpressionPosition.UNKNOWN), Types.required(MinorType.VARCHAR));
-    VarCharVector vector = new VarCharVector(f, bee.getContext().getAllocator());
-    byte[] bytes = value.getBytes(Charsets.UTF_8);
-    vector.allocateNew(bytes.length, 1);
-    vector.getMutator().set(0, bytes);
-    vector.getMutator().setValueCount(1);
-    QueryResult header = QueryResult.newBuilder() //
-        .setQueryId(context.getQueryId()) //
-        .setRowCount(1) //
-        .setDef(RecordBatchDef.newBuilder().addField(vector.getMetadata()).setRecordCount(1).build()) //
-        .setIsLastChunk(false) //
-        .build();
-    QueryWritableBatch b1 = new QueryWritableBatch(header, vector.getBuffers());
-    vector.close();
-    
-    QueryResult header2 = QueryResult.newBuilder() //
-        .setQueryId(context.getQueryId()) //
-        .setRowCount(0) //
-        .setDef(RecordBatchDef.getDefaultInstance()) //
-        .setIsLastChunk(true) //
-        .build();
-    QueryWritableBatch b2 = new QueryWritableBatch(header2);
-    
-    SingleListener l = new SingleListener();
-    this.initiatingClient.sendResult(l, b1);
-    this.initiatingClient.sendResult(l, b2);
-    l.acct.waitForSendComplete();
-    
+
+  private class PhysicalFromLogicalExplain{
+    public String json;
+
+    public PhysicalFromLogicalExplain(String json) {
+      super();
+      this.json = json;
+    }
+
   }
-  
-  
+
   class SingleListener implements RpcOutcomeListener<Ack>{
 
     final SendingAccountor acct;
-    
+
     public SingleListener(){
       acct  = new SendingAccountor();
       acct.increment();
       acct.increment();
     }
-    
+
     @Override
     public void failed(RpcException ex) {
       acct.decrement();
@@ -271,11 +246,11 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
     public void success(Ack value, ByteBuf buffer) {
       acct.decrement();
     }
-    
+
   }
-  
 
-  
+
+
   private void parseAndRunPhysicalPlan(String json) {
     try {
       PhysicalPlan plan = context.getPlanReader().readPhysicalPlan(json);
@@ -291,7 +266,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       fail(String.format("Failure running plan.  You requested a result mode of %s and a physical plan can only be output as EXEC", plan.getProperties().resultMode), new Exception());
     }
     PhysicalOperator rootOperator = plan.getSortedOperators(false).iterator().next();
-    
+
     MakeFragmentsVisitor makeFragmentsVisitor = new MakeFragmentsVisitor();
     Fragment rootFragment;
     try {
@@ -300,10 +275,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       fail("Failure while fragmenting query.", e);
       return;
     }
-    
-    
 
-    
     PlanningSet planningSet = StatsCollector.collectStats(rootFragment);
     SimpleParallelizer parallelizer = new SimpleParallelizer();
 
@@ -319,26 +291,26 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
       // store fragments in distributed grid.
       logger.debug("Storing fragments");
       for (PlanFragment f : work.getFragments()) {
-        
+
         // store all fragments in grid since they are part of handshake.
-        
+
         context.getCache().storeFragment(f);
         if (f.getLeafFragment()) {
           leafFragments.add(f);
         } else {
           intermediateFragments.add(f);
         }
-        
-        
+
+
       }
 
       logger.debug("Fragments stored.");
-      
+
       logger.debug("Submitting fragments to run.");
       fragmentManager.runFragments(bee, work.getRootFragment(), work.getRootOperator(), initiatingClient, leafFragments, intermediateFragments);
       logger.debug("Fragments running.");
 
-    
+
     } catch (ExecutionSetupException | RpcException e) {
       fail("Failure while setting up query.", e);
     }
@@ -347,29 +319,9 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
 
   private void runSQL(String sql) {
     try{
-      DrillSqlWorker sqlWorker = new DrillSqlWorker(context.getFactory(), context.getFunctionRegistry());
-      
-      RelResult relResult = sqlWorker.getLogicalRel(sql);
-      
-      //EXPLAIN logical
-      if (relResult.getMode() == ResultMode.LOGICAL) {
-        returnLogical(sqlWorker.getLogicalPlan(relResult));
-        return;
-      }
-      
-      PhysicalPlan physical = sqlWorker.getPhysicalPlan(relResult, context);
-            
-      if(logger.isDebugEnabled()) {
-        logger.debug("Distributed Physical {}", context.getConfig().getMapper().writeValueAsString(physical));
-      }
-      
-      //EXPLAIN physical
-      if (relResult.getMode() == ResultMode.PHYSICAL) {
-        returnPhysical(physical);
-        return;
-      }
-      
-      runPhysicalPlan(physical);
+      DrillSqlWorker sqlWorker = new DrillSqlWorker(context);
+      PhysicalPlan plan = sqlWorker.getPlan(sql);
+      runPhysicalPlan(plan);
     }catch(Exception e){
       fail("Failure while parsing sql.", e);
     }
@@ -392,21 +344,21 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   @Override
   public void close() throws IOException {
   }
-  
+
   QueryState getQueryState(){
     return this.state.getState();
   }
 
-  
+
   class ForemanManagerListener{
     void fail(String message, Throwable t) {
       ForemanManagerListener.this.fail(message, t);
     }
-    
+
     void cleanupAndSendResult(QueryResult result){
       Foreman.this.cleanupAndSendResult(result);
     }
-    
+
   }
 
 
@@ -415,7 +367,7 @@ public class Foreman implements Runnable, Closeable, Comparable<Object>{
   public int compareTo(Object o) {
     return o.hashCode() - o.hashCode();
   }
-  
-  
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
index 9206205..2b8779a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/work/user/UserWorker.java
@@ -28,15 +28,16 @@ import org.apache.drill.exec.proto.UserProtos.RequestResults;
 import org.apache.drill.exec.proto.UserProtos.RunQuery;
 import org.apache.drill.exec.rpc.Acks;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.drill.exec.work.WorkManager.WorkerBee;
 import org.apache.drill.exec.work.foreman.Foreman;
 import org.apache.drill.exec.work.fragment.FragmentExecutor;
 
 public class UserWorker{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(UserWorker.class);
-  
+
   private final WorkerBee bee;
-    
+
   public UserWorker(WorkerBee bee) {
     super();
     this.bee = bee;
@@ -49,7 +50,7 @@ public class UserWorker{
     bee.addNewForeman(foreman);
     return id;
   }
-  
+
   public QueryResult getResult(UserClientConnection connection, RequestResults req){
     Foreman foreman = bee.getForemanForQueryId(req.getQueryId());
     if(foreman == null) return QueryResult.newBuilder().setQueryState(QueryState.UNKNOWN_QUERY).build();
@@ -63,10 +64,14 @@ public class UserWorker{
     }
     return Acks.OK;
   }
-  
+
   public Ack cancelFragment(FragmentHandle handle){
     FragmentExecutor runner = bee.getFragmentRunner(handle);
     if(runner != null) runner.cancel();
     return Acks.OK;
   }
+
+  public SchemaFactory getSchemaFactory(){
+    return bee.getContext().getSchemaFactory();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 0e17f8e..8b85ac2 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
@@ -19,60 +19,135 @@ package org.apache.drill;
 
 import java.io.IOException;
 import java.net.URL;
+import java.util.List;
 
+import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.util.TestTools;
+import org.apache.drill.common.util.TestTools.TestLogReporter;
+import org.apache.drill.exec.client.DrillClient;
+import org.apache.drill.exec.client.PrintingResultsListener;
 import org.apache.drill.exec.client.QuerySubmitter;
-import org.apache.drill.exec.store.ResourceInputStream;
+import org.apache.drill.exec.client.QuerySubmitter.Format;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.server.Drillbit;
+import org.apache.drill.exec.server.RemoteServiceSet;
+import org.apache.drill.exec.util.VectorUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
 import org.junit.Rule;
 import org.junit.rules.TestRule;
+import org.junit.rules.TestWatcher;
+import org.junit.runner.Description;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.Resources;
 
 public class BaseTestQuery {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
-  
+
+  // make it static so we can use after class
+  static final TestLogReporter LOG_OUTCOME = TestTools.getTestLogReporter(logger);
+
   @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(20000);
-  
-  protected void test(String sql) throws Exception{
-    boolean good = false;
-    sql = sql.replace("[WORKING_PATH]", TestTools.getWorkingPath());
-    
-    try{
-      QuerySubmitter s = new QuerySubmitter();
-      s.submitQuery(null, sql, "sql", null, true, 1, "tsv");
-      good = true;
-    }finally{
-      if(!good) Thread.sleep(2000);
-    }
+  @Rule public final TestLogReporter logOutcome = LOG_OUTCOME;
+
+  @AfterClass
+  public static void letLogsCatchUp() throws InterruptedException{
+    LOG_OUTCOME.sleepIfFailure();
   }
-  
-  protected void testLogical(String logical) throws Exception{
-    boolean good = false;
-    logical = logical.replace("[WORKING_PATH]", TestTools.getWorkingPath());
-    
-    try{
-      QuerySubmitter s = new QuerySubmitter();
-      s.submitQuery(null, logical, "logical", null, true, 1, "tsv");
-      good = true;
-    }finally{
-      if(!good) Thread.sleep(2000);
+
+  public final TestRule resetWatcher = new TestWatcher() {
+    @Override
+    protected void failed(Throwable e, Description description) {
+      try {
+        resetClientAndBit();
+      } catch (Exception e1) {
+        throw new RuntimeException("Failure while resetting client.", e1);
+      }
     }
+  };
+
+  static DrillClient client;
+  static Drillbit bit;
+  static RemoteServiceSet serviceSet;
+  static DrillConfig config;
+  static QuerySubmitter submitter = new QuerySubmitter();
+
+  static void resetClientAndBit() throws Exception{
+    closeClient();
+    openClient();
+  }
+
+  @BeforeClass
+  public static void openClient() throws Exception{
+    config = DrillConfig.create();
+    serviceSet = RemoteServiceSet.getLocalServiceSet();
+    bit = new Drillbit(config, serviceSet);
+    bit.run();
+    client = new DrillClient(config, serviceSet.getCoordinator());
+    client.connect();
+  }
+
+  protected BufferAllocator getAllocator(){
+    return client.getAllocator();
+  }
+
+  @AfterClass
+  public static void closeClient() throws IOException{
+    if(client != null) client.close();
+    if(bit != null) bit.close();
+    if(serviceSet != null) serviceSet.close();
+  }
+
+
+  protected List<QueryResultBatch> testSqlWithResults(String sql) throws Exception{
+    return testRunAndReturn(QueryType.SQL, sql);
+  }
+
+  protected List<QueryResultBatch> testLogicalWithResults(String logical) throws Exception{
+    return testRunAndReturn(QueryType.LOGICAL, logical);
+  }
+
+  protected List<QueryResultBatch> testPhysicalWithResults(String physical) throws Exception{
+    return testRunAndReturn(QueryType.PHYSICAL, physical);
+  }
+
+  private List<QueryResultBatch>  testRunAndReturn(QueryType type, String query) throws Exception{
+    query = query.replace("[WORKING_PATH]", TestTools.getWorkingPath());
+    return client.runQuery(type, query);
+  }
+
+  private int testRunAndPrint(QueryType type, String query) throws Exception{
+    query = query.replace("[WORKING_PATH]", TestTools.getWorkingPath());
+    PrintingResultsListener resultListener = new PrintingResultsListener(Format.TSV, VectorUtil.DEFAULT_COLUMN_WIDTH);
+    client.runQuery(type, query, resultListener);
+    return resultListener.await();
+  }
+
+  protected void testWithListener(QueryType type, String query, UserResultsListener resultListener){
+    query = query.replace("[WORKING_PATH]", TestTools.getWorkingPath());
+    client.runQuery(type, query, resultListener);
   }
-  
-  protected void testPhysical(String physical) throws Exception{
-    boolean good = false;
-    physical = physical.replace("[WORKING_PATH]", TestTools.getWorkingPath());
-    
-    try{
-      QuerySubmitter s = new QuerySubmitter();
-      s.submitQuery(null, physical, "physical", null, true, 1, "tsv");
-      good = true;
-    }finally{
-      if(!good) Thread.sleep(2000);
+
+  protected void test(String query) throws Exception{
+    String[] queries = query.split(";");
+    for(String q : queries){
+      if(q.trim().isEmpty()) continue;
+      testRunAndPrint(QueryType.SQL, q);
     }
   }
-  
+
+  protected int testLogical(String query) throws Exception{
+    return testRunAndPrint(QueryType.LOGICAL, query);
+  }
+
+  protected int testPhysical(String query) throws Exception{
+    return testRunAndPrint(QueryType.PHYSICAL, query);
+  }
+
   protected void testPhysicalFromFile(String file) throws Exception{
     testPhysical(getFile(file));
   }
@@ -82,7 +157,8 @@ public class BaseTestQuery {
   protected void testSqlFromFile(String file) throws Exception{
     test(getFile(file));
   }
-  
+
+
   protected String getFile(String resource) throws IOException{
     URL url = Resources.getResource(resource);
     if(url == null){

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
new file mode 100644
index 0000000..d719bc5
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/PlanningBase.java
@@ -0,0 +1,114 @@
+/**
+ * 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;
+
+import java.io.IOException;
+import java.net.URL;
+
+import mockit.Mocked;
+import mockit.NonStrictExpectations;
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.tools.Frameworks;
+
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DrillSqlWorker;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.rpc.user.UserSession;
+import org.apache.drill.exec.server.DrillbitContext;
+import org.apache.drill.exec.store.StoragePluginRegistry;
+import org.junit.Rule;
+import org.junit.rules.TestRule;
+
+import com.codahale.metrics.MetricRegistry;
+import com.google.common.base.Charsets;
+import com.google.common.collect.ImmutableList;
+import com.google.common.io.Resources;
+
+public class PlanningBase {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningBase.class);
+
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(30000);
+  @Rule public final TestRule LOG_OUTCOME = TestTools.getTestLogReporter(logger);
+
+  @Mocked DrillbitContext dbContext;
+  @Mocked QueryContext context;
+  private final DrillConfig config = DrillConfig.create();
+
+
+  protected void testSqlPlanFromFile(String file) throws Exception{
+    testSqlPlan(getFile(file));
+  }
+
+  protected void testSqlPlan(String sqlCommands) throws Exception{
+    String[] sqlStrings = sqlCommands.split(";");
+
+    new NonStrictExpectations() {
+      {
+        dbContext.getMetrics();
+        result = new MetricRegistry();
+        dbContext.getAllocator();
+        result = new TopLevelAllocator();
+        dbContext.getConfig();
+        result = config;
+      }
+    };
+
+    StoragePluginRegistry registry = new StoragePluginRegistry(dbContext);
+    final FunctionImplementationRegistry functionRegistry = new FunctionImplementationRegistry(config);
+    final SchemaPlus root = Frameworks.createRootSchema();
+    registry.getSchemaFactory().registerSchemas(null, root);
+
+
+    new NonStrictExpectations() {
+      {
+        context.getNewDefaultSchema();
+        result = root;
+        context.getFunctionRegistry();
+        result = functionRegistry;
+        context.getSession();
+        result = new UserSession(null, null, null);
+        context.getCurrentEndpoint();
+        result = DrillbitEndpoint.getDefaultInstance();
+        context.getActiveEndpoints();
+        result = ImmutableList.of(DrillbitEndpoint.getDefaultInstance());
+        context.getConfig();
+        result = config;
+      }
+    };
+
+    for(String sql : sqlStrings){
+      if(sql.trim().isEmpty()) continue;
+      DrillSqlWorker worker = new DrillSqlWorker(context);
+      PhysicalPlan p = worker.getPlan(sql);
+    }
+
+  }
+
+  protected String getFile(String resource) throws IOException{
+    URL url = Resources.getResource(resource);
+    if(url == null){
+      throw new IOException(String.format("Unable to find path %s.", resource));
+    }
+    return Resources.toString(url, Charsets.UTF_8);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
index 0521fca..5b3a11b 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
@@ -23,25 +23,23 @@ import org.junit.Rule;
 import org.junit.Test;
 import org.junit.rules.TestRule;
 
-public class TestAltSortQueries {
+public class TestAltSortQueries extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAltSortQueries.class);
-  
-  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(10000000);
 
   @Test
   public void testOrderBy() throws Exception{
     test("select R_REGIONKEY " +
          "from dfs.`[WORKING_PATH]/../../sample-data/region.parquet` " +
-         "order by R_REGIONKEY");   
-  }  
+         "order by R_REGIONKEY");
+  }
 
-  @Test 
+  @Test
   public void testOrderBySingleFile() throws Exception{
     test("select R_REGIONKEY " +
          "from dfs.`[WORKING_PATH]/../../sample-data/regionsSF/` " +
-         "order by R_REGIONKEY");   
-  }  
-    
+         "order by R_REGIONKEY");
+  }
+
   @Test
   public void testSelectWithLimit() throws Exception{
     test("select employee_id,  first_name, last_name from cp.`employee.json` order by employee_id limit 5 ");
@@ -54,30 +52,16 @@ public class TestAltSortQueries {
 
   @Test
   public void testJoinWithLimit() throws Exception{
-    test("SELECT\n" + 
-        "  nations.N_NAME,\n" + 
-        "  regions.R_NAME\n" + 
-        "FROM\n" + 
-        "  dfs.`[WORKING_PATH]/../../sample-data/nation.parquet` nations\n" + 
-        "JOIN\n" + 
-        "  dfs.`[WORKING_PATH]/../../sample-data/region.parquet` regions\n" + 
+    test("SELECT\n" +
+        "  nations.N_NAME,\n" +
+        "  regions.R_NAME\n" +
+        "FROM\n" +
+        "  dfs.`[WORKING_PATH]/../../sample-data/nation.parquet` nations\n" +
+        "JOIN\n" +
+        "  dfs.`[WORKING_PATH]/../../sample-data/region.parquet` regions\n" +
         "  on nations.N_REGIONKEY = regions.R_REGIONKEY" +
-        " order by regions.R_NAME, nations.N_NAME " + 
+        " order by regions.R_NAME, nations.N_NAME " +
         " limit 5");
   }
-    
-  
-  private void test(String sql) throws Exception{
-    boolean good = false;
-    sql = sql.replace("[WORKING_PATH]", TestTools.getWorkingPath());
-    
-    try{
-      QuerySubmitter s = new QuerySubmitter();
-      s.submitQuery(null, sql, "sql", null, true, 1, "tsv");
-      good = true;
-    }finally{
-      if(!good) Thread.sleep(2000);
-    }
-  }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 4f179a0..cbf19ac 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,39 +17,32 @@
  */
 package org.apache.drill;
 
-import org.apache.drill.common.util.TestTools;
-import org.apache.drill.exec.client.QuerySubmitter;
-import org.junit.Ignore;
-import org.junit.Rule;
 import org.junit.Test;
-import org.junit.rules.TestRule;
 
-public class TestExampleQueries {
+public class TestExampleQueries extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestExampleQueries.class);
-  
-  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(10000000);
-  
+
   @Test
   public void testSelectWithLimit() throws Exception{
     test("select employee_id,  first_name, last_name from cp.`employee.json` order by employee_id limit 5 offset 10");
   }
-  
+
   @Test
   public void testJoin() throws Exception{
-    test("SELECT\n" + 
-        "  nations.N_NAME,\n" + 
-        "  regions.R_NAME\n" + 
-        "FROM\n" + 
-        "  dfs.`[WORKING_PATH]/../../sample-data/nation.parquet` nations\n" + 
-        "JOIN\n" + 
-        "  dfs.`[WORKING_PATH]/../../sample-data/region.parquet` regions\n" + 
+    test("SELECT\n" +
+        "  nations.N_NAME,\n" +
+        "  regions.R_NAME\n" +
+        "FROM\n" +
+        "  dfs.`[WORKING_PATH]/../../sample-data/nation.parquet` nations\n" +
+        "JOIN\n" +
+        "  dfs.`[WORKING_PATH]/../../sample-data/region.parquet` regions\n" +
         "  on nations.N_REGIONKEY = regions.R_REGIONKEY");
   }
-  
-  
+
+
   @Test
   public void testWhere() throws Exception{
-    test("select * from cp.`employee.json` where employee_id > 10 and employee_id < 20");
+    test("select * from cp.`employee.json` ");
   }
 
   @Test
@@ -58,26 +51,13 @@ public class TestExampleQueries {
   }
 
   @Test
-  public void testExpalinPhysical() throws Exception{
+  public void testExplainPhysical() throws Exception{
     test("explain plan for select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
   }
 
   @Test
-  public void testExpalinLogical() throws Exception{
+  public void testExplainLogical() throws Exception{
     test("explain plan without implementation for select marital_status, COUNT(1) as cnt from cp.`employee.json` group by marital_status");
   }
-  
-  private void test(String sql) throws Exception{
-    boolean good = false;
-    sql = sql.replace("[WORKING_PATH]", TestTools.getWorkingPath());
-    
-    try{
-      QuerySubmitter s = new QuerySubmitter();
-      s.submitQuery(null, sql, "sql", null, true, 1, "tsv");
-      good = true;
-    }finally{
-      if(!good) Thread.sleep(2000);
-    }
-  }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
new file mode 100644
index 0000000..7a6982d
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchDistributed.java
@@ -0,0 +1,154 @@
+/**
+ * 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;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestTpchDistributed extends BaseTestQuery{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchDistributed.class);
+
+  private void testDistributed(String fileName) throws Exception{
+    String query = getFile(fileName);
+    test(query);
+  }
+
+  @Test
+  public void tpch01() throws Exception{
+    testDistributed("queries/tpch/01.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-512
+  public void tpch02() throws Exception{
+    testDistributed("queries/tpch/02.sql");
+  }
+
+  @Test
+  public void tpch03() throws Exception{
+    testDistributed("queries/tpch/03.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-534
+  public void tpch04() throws Exception{
+    testDistributed("queries/tpch/04.sql");
+  }
+
+  @Test
+  public void tpch05() throws Exception{
+    testDistributed("queries/tpch/05.sql");
+  }
+
+  @Test
+  public void tpch06() throws Exception{
+    testDistributed("queries/tpch/06.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch07() throws Exception{
+    testDistributed("queries/tpch/07.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch08() throws Exception{
+    testDistributed("queries/tpch/08.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch09() throws Exception{
+    testDistributed("queries/tpch/09.sql");
+  }
+
+  @Test
+  public void tpch10() throws Exception{
+    testDistributed("queries/tpch/10.sql");
+  }
+
+  @Test
+  @Ignore // cartesion problem
+  public void tpch11() throws Exception{
+    testDistributed("queries/tpch/11.sql");
+  }
+
+  @Test
+  public void tpch12() throws Exception{
+    testDistributed("queries/tpch/12.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-548 flapping test: issues with writerIndex.
+  public void tpch13() throws Exception{
+    testDistributed("queries/tpch/13.sql");
+  }
+
+  @Test
+  public void tpch14() throws Exception{
+    testDistributed("queries/tpch/14.sql");
+  }
+
+  @Test
+  @Ignore // requires views.
+  public void tpch15() throws Exception{
+    testDistributed("queries/tpch/15.sql");
+  }
+
+  @Test
+  public void tpch16() throws Exception{
+    testDistributed("queries/tpch/16.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch17() throws Exception{
+    testDistributed("queries/tpch/17.sql");
+  }
+
+  @Test
+  public void tpch18() throws Exception{
+    testDistributed("queries/tpch/18.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch19() throws Exception{
+    testDistributed("queries/tpch/19.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch20() throws Exception{
+    testDistributed("queries/tpch/20.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch21() throws Exception{
+    testDistributed("queries/tpch/21.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-518
+  public void tpch22() throws Exception{
+    testDistributed("queries/tpch/22.sql");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
new file mode 100644
index 0000000..c64f330
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchExplain.java
@@ -0,0 +1,155 @@
+/**
+ * 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;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestTpchExplain extends BaseTestQuery{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchExplain.class);
+
+  private static final String EXPLAIN_PREFIX = "EXPLAIN PLAN FOR ";
+
+
+  private void doExplain(String fileName) throws Exception{
+    String query = getFile(fileName);
+    query = EXPLAIN_PREFIX + query;
+    test(query);
+  }
+
+  @Test
+  public void tpch01() throws Exception{
+    doExplain("queries/tpch/01.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-512
+  public void tpch02() throws Exception{
+    doExplain("queries/tpch/02.sql");
+  }
+
+  @Test
+  public void tpch03() throws Exception{
+    doExplain("queries/tpch/03.sql");
+  }
+
+  @Test
+  public void tpch04() throws Exception{
+    doExplain("queries/tpch/04.sql");
+  }
+
+  @Test
+  public void tpch05() throws Exception{
+    doExplain("queries/tpch/05.sql");
+  }
+
+  @Test
+  public void tpch06() throws Exception{
+    doExplain("queries/tpch/06.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch07() throws Exception{
+    doExplain("queries/tpch/07.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch08() throws Exception{
+    doExplain("queries/tpch/08.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch09() throws Exception{
+    doExplain("queries/tpch/09.sql");
+  }
+
+  @Test
+  public void tpch10() throws Exception{
+    doExplain("queries/tpch/10.sql");
+  }
+
+  @Test
+  @Ignore // cartesion problem
+  public void tpch11() throws Exception{
+    doExplain("queries/tpch/11.sql");
+  }
+
+  @Test
+  public void tpch12() throws Exception{
+    doExplain("queries/tpch/12.sql");
+  }
+
+  @Test
+  public void tpch13() throws Exception{
+    doExplain("queries/tpch/13.sql");
+  }
+
+  @Test
+  public void tpch14() throws Exception{
+    doExplain("queries/tpch/14.sql");
+  }
+
+  @Test
+  @Ignore // requires views.
+  public void tpch15() throws Exception{
+    doExplain("queries/tpch/15.sql");
+  }
+
+  @Test
+  public void tpch16() throws Exception{
+    doExplain("queries/tpch/16.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch17() throws Exception{
+    doExplain("queries/tpch/17.sql");
+  }
+
+  @Test
+  public void tpch18() throws Exception{
+    doExplain("queries/tpch/18.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch19() throws Exception{
+    doExplain("queries/tpch/19.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch20() throws Exception{
+    doExplain("queries/tpch/20.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch21() throws Exception{
+    doExplain("queries/tpch/21.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-518
+  public void tpch22() throws Exception{
+    doExplain("queries/tpch/22.sql");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
new file mode 100644
index 0000000..7327c67
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchPlanning.java
@@ -0,0 +1,150 @@
+/**
+ * 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;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+
+public class TestTpchPlanning extends PlanningBase{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchPlanning.class);
+
+
+
+  @Test
+  public void tpch01() throws Exception{
+    testSqlPlanFromFile("queries/tpch/01.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-512
+  public void tpch02() throws Exception{
+    testSqlPlanFromFile("queries/tpch/02.sql");
+  }
+
+  @Test
+  public void tpch03() throws Exception{
+    testSqlPlanFromFile("queries/tpch/03.sql");
+  }
+
+  @Test
+  public void tpch04() throws Exception{
+    testSqlPlanFromFile("queries/tpch/04.sql");
+  }
+
+  @Test
+  public void tpch05() throws Exception{
+    testSqlPlanFromFile("queries/tpch/05.sql");
+  }
+
+  @Test
+  public void tpch06() throws Exception{
+    testSqlPlanFromFile("queries/tpch/06.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch07() throws Exception{
+    testSqlPlanFromFile("queries/tpch/07.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch08() throws Exception{
+    testSqlPlanFromFile("queries/tpch/08.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch09() throws Exception{
+    testSqlPlanFromFile("queries/tpch/09.sql");
+  }
+
+  @Test
+  public void tpch10() throws Exception{
+    testSqlPlanFromFile("queries/tpch/10.sql");
+  }
+
+  @Test
+  @Ignore // cartesion problem
+  public void tpch11() throws Exception{
+    testSqlPlanFromFile("queries/tpch/11.sql");
+  }
+
+  @Test
+  public void tpch12() throws Exception{
+    testSqlPlanFromFile("queries/tpch/12.sql");
+  }
+
+  @Test
+  public void tpch13() throws Exception{
+    testSqlPlanFromFile("queries/tpch/13.sql");
+  }
+
+  @Test
+  public void tpch14() throws Exception{
+    testSqlPlanFromFile("queries/tpch/14.sql");
+  }
+
+  @Test
+  @Ignore // requires views.
+  public void tpch15() throws Exception{
+    testSqlPlanFromFile("queries/tpch/15.sql");
+  }
+
+  @Test
+  public void tpch16() throws Exception{
+    testSqlPlanFromFile("queries/tpch/16.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch17() throws Exception{
+    testSqlPlanFromFile("queries/tpch/17.sql");
+  }
+
+  @Test
+  public void tpch18() throws Exception{
+    testSqlPlanFromFile("queries/tpch/18.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch19() throws Exception{
+    testSqlPlanFromFile("queries/tpch/19.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch20() throws Exception{
+    testSqlPlanFromFile("queries/tpch/20.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch21() throws Exception{
+    testSqlPlanFromFile("queries/tpch/21.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-518
+  public void tpch22() throws Exception{
+    testSqlPlanFromFile("queries/tpch/22.sql");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
deleted file mode 100644
index fada2f0..0000000
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
+++ /dev/null
@@ -1,158 +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;
-
-import org.junit.Ignore;
-import org.junit.Test;
-
-public class TestTpchQueries extends BaseTestQuery{
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchQueries.class);
-  
-  @Test
-  @Ignore
-  public void tpch01() throws Exception{
-    testSqlFromFile("queries/tpch/01.sql");
-  }
-  
-  @Test
-  @Ignore // DRILL-435
-  public void tpch02() throws Exception{
-    testSqlFromFile("queries/tpch/02.sql");
-  }
-
-  @Test
-  @Ignore
-  public void tpch03() throws Exception{
-    testSqlFromFile("queries/tpch/03.sql");
-  }
-
-  @Test
-  @Ignore // DRILL-437
-  public void tpch04() throws Exception{
-    testSqlFromFile("queries/tpch/04.sql");
-  }
-  
-  @Test 
-  @Ignore // DRILL-436
-  public void tpch05() throws Exception{
-    testSqlFromFile("queries/tpch/05.sql");
-  }
-  
-  @Test  // DRILL-356
-  @Ignore
-  public void tpch06() throws Exception{
-    testSqlFromFile("queries/tpch/06.sql");
-  }
-
-  @Test 
-  @Ignore // DRILL-439
-  public void tpch07() throws Exception{
-    testSqlFromFile("queries/tpch/07.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-356
-  public void tpch08() throws Exception{
-    testSqlFromFile("queries/tpch/08.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-435
-  public void tpch09() throws Exception{
-    testSqlFromFile("queries/tpch/09.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-356  
-  public void tpch10() throws Exception{
-    testSqlFromFile("queries/tpch/10.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-436
-  public void tpch11() throws Exception{
-    testSqlFromFile("queries/tpch/11.sql");
-  }
-
-  @Test 
-  @Ignore // DRILL-403
-  public void tpch12() throws Exception{
-    testSqlFromFile("queries/tpch/12.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-435
-  public void tpch13() throws Exception{
-    testSqlFromFile("queries/tpch/13.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-435
-  public void tpch14() throws Exception{
-    testSqlFromFile("queries/tpch/14.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-438
-  public void tpch15() throws Exception{
-    testSqlFromFile("queries/tpch/15.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-435
-  public void tpch16() throws Exception{
-    testSqlFromFile("queries/tpch/16.sql");
-  }
-
-  @Test
-  @Ignore // DRILL-440
-  public void tpch17() throws Exception{
-    testSqlFromFile("queries/tpch/17.sql");
-  }
-
-  @Test 
-  @Ignore // DRILL-436
-  public void tpch18() throws Exception{
-    testSqlFromFile("queries/tpch/18.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-436
-  public void tpch19() throws Exception{
-    testSqlFromFile("queries/tpch/19.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-435
-  public void tpch20() throws Exception{
-    testSqlFromFile("queries/tpch/20.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-440
-  public void tpch21() throws Exception{
-    testSqlFromFile("queries/tpch/21.sql");
-  }
-
-  @Test  
-  @Ignore // DRILL-441
-  public void tpch22() throws Exception{
-    testSqlFromFile("queries/tpch/22.sql");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
new file mode 100644
index 0000000..1ccb65c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchSingleMode.java
@@ -0,0 +1,157 @@
+/**
+ * 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;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestTpchSingleMode extends BaseTestQuery{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTpchSingleMode.class);
+
+  private static final String SINGLE_MODE = "ALTER SESSION SET NO_EXCHANGES = true;";
+
+
+  private void testSingleMode(String fileName) throws Exception{
+    String query = getFile(fileName);
+    query = SINGLE_MODE + query;
+    test(query);
+  }
+
+  @Test
+  public void tpch01() throws Exception{
+    testSingleMode("queries/tpch/01.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-512
+  public void tpch02() throws Exception{
+    testSingleMode("queries/tpch/02.sql");
+  }
+
+  @Test
+  public void tpch03() throws Exception{
+    testSingleMode("queries/tpch/03.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-534
+  public void tpch04() throws Exception{
+    testSingleMode("queries/tpch/04.sql");
+  }
+
+  @Test
+  public void tpch05() throws Exception{
+    testSingleMode("queries/tpch/05.sql");
+  }
+
+  @Test
+  public void tpch06() throws Exception{
+    testSingleMode("queries/tpch/06.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch07() throws Exception{
+    testSingleMode("queries/tpch/07.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch08() throws Exception{
+    testSingleMode("queries/tpch/08.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-516
+  public void tpch09() throws Exception{
+    testSingleMode("queries/tpch/09.sql");
+  }
+
+  @Test
+  public void tpch10() throws Exception{
+    testSingleMode("queries/tpch/10.sql");
+  }
+
+  @Test
+  @Ignore // Cartesian problem
+  public void tpch11() throws Exception{
+    testSingleMode("queries/tpch/11.sql");
+  }
+
+  @Test
+  public void tpch12() throws Exception{
+    testSingleMode("queries/tpch/12.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-548 flapping test: issues with writerIndex.
+  public void tpch13() throws Exception{
+    testSingleMode("queries/tpch/13.sql");
+  }
+
+  @Test
+  public void tpch14() throws Exception{
+    testSingleMode("queries/tpch/14.sql");
+  }
+
+  @Test
+  @Ignore // requires views.
+  public void tpch15() throws Exception{
+    testSingleMode("queries/tpch/15.sql");
+  }
+
+  @Test
+  public void tpch16() throws Exception{
+    testSingleMode("queries/tpch/16.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch17() throws Exception{
+    testSingleMode("queries/tpch/17.sql");
+  }
+
+  @Test
+  public void tpch18() throws Exception{
+    testSingleMode("queries/tpch/18.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch19() throws Exception{
+    testSingleMode("queries/tpch/19.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-517
+  public void tpch20() throws Exception{
+    testSingleMode("queries/tpch/20.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-519
+  public void tpch21() throws Exception{
+    testSingleMode("queries/tpch/21.sql");
+  }
+
+  @Test
+  @Ignore // DRILL-518
+  public void tpch22() throws Exception{
+    testSingleMode("queries/tpch/22.sql");
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
index cf274c1..158d4d7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/cache/TestVectorCache.java
@@ -46,7 +46,7 @@ public class TestVectorCache {
     HazelCache cache = new HazelCache(config, context.getAllocator());
     cache.run();
 
-    MaterializedField intField = MaterializedField.create(new SchemaPath("int", ExpressionPosition.UNKNOWN), Types.required(TypeProtos.MinorType.INT));
+    MaterializedField intField = MaterializedField.create(SchemaPath.getSimplePath("int"), Types.required(TypeProtos.MinorType.INT));
     IntVector intVector = (IntVector)TypeHelper.getNewVector(intField, context.getAllocator());
     MaterializedField binField = MaterializedField.create(new SchemaPath("binary", ExpressionPosition.UNKNOWN), Types.required(TypeProtos.MinorType.VARBINARY));
     VarBinaryVector binVector = (VarBinaryVector)TypeHelper.getNewVector(binField, context.getAllocator());

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
index 4d09b1d..f2be81f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/SimpleRootExec.java
@@ -33,24 +33,25 @@ import org.apache.drill.exec.vector.ValueVector;
 
 import com.beust.jcommander.internal.Lists;
 
+@Deprecated
 public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleRootExec.class);
 
   private RecordBatch incoming;
-  
+
   public SimpleRootExec(RootExec e){
     if(e instanceof ScreenRoot){
-      incoming = ((ScreenRoot)e).getIncoming();  
+      incoming = ((ScreenRoot)e).getIncoming();
     }else{
       throw new UnsupportedOperationException();
     }
-    
+
   }
 
   public FragmentContext getContext(){
     return incoming.getContext();
   }
-  
+
   public SelectionVector2 getSelectionVector2(){
     return incoming.getSelectionVector2();
   }
@@ -64,7 +65,7 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
     TypedFieldId tfid = incoming.getValueVectorId(path);
     return (T) incoming.getValueAccessorById(tfid.getFieldId(), vvClass).getValueVector();
   }
-  
+
   @Override
   public boolean next() {
     switch(incoming.next()){
@@ -94,10 +95,10 @@ public class SimpleRootExec implements RootExec, Iterable<ValueVector>{
   public int getRecordCount(){
     return incoming.getRecordCount();
   }
-  
+
   /// Temporary: for exposing the incoming batch to TestHashTable
   public RecordBatch getIncoming() {
 	  return incoming;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
index b55681d..69ba832 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestBroadcastExchange.java
@@ -54,8 +54,8 @@ public class TestBroadcastExchange extends PopUnitTestBase {
       List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, physicalPlan);
       int count = 0;
       for(QueryResultBatch b : results) {
-        if (b.getHeader().getRowCount() != 0)
-          count += b.getHeader().getRowCount();
+        if (b.getHeader().getRowCount() != 0) count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(25, count);
     }
@@ -78,8 +78,8 @@ public class TestBroadcastExchange extends PopUnitTestBase {
       List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL, physicalPlan);
       int count = 0;
       for(QueryResultBatch b : results) {
-        if (b.getHeader().getRowCount() != 0)
-          count += b.getHeader().getRowCount();
+        if (b.getHeader().getRowCount() != 0) count += b.getHeader().getRowCount();
+        b.release();
       }
       System.out.println(count);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
index 0659e5b..ab792cd 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastFunctions.java
@@ -37,6 +37,7 @@ import org.apache.drill.exec.expr.holders.Float8Holder;
 import org.apache.drill.exec.expr.holders.IntHolder;
 import org.apache.drill.exec.expr.holders.VarBinaryHolder;
 import org.apache.drill.exec.expr.holders.VarCharHolder;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
@@ -58,12 +59,9 @@ import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.vector.Float4Vector;
 import org.apache.drill.exec.vector.Float8Vector;
 import org.apache.drill.exec.vector.IntVector;
-import org.apache.drill.exec.vector.NullableVarCharVector;
-import org.apache.drill.exec.vector.ValueVector;
 import org.apache.drill.exec.vector.VarBinaryVector;
 import org.apache.drill.exec.vector.VarCharVector;
 import org.junit.After;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import com.codahale.metrics.MetricRegistry;
@@ -80,9 +78,12 @@ public class TestCastFunctions extends PopUnitTestBase{
   // cast to bigint.
   public void testCastBigInt(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
+
+    final BufferAllocator allocator = new TopLevelAllocator();
+
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -92,23 +93,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       BigIntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), BigIntVector.class);
       BigIntVector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           BigIntHolder holder0 = new BigIntHolder();
           a0.get(i, holder0);
           assertEquals(1256, holder0.value);
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -120,9 +123,11 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastInt(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
+
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -132,23 +137,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       IntVector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast", ExpressionPosition.UNKNOWN), IntVector.class);
       IntVector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           IntHolder holder0 = new IntHolder();
           a0.get(i, holder0);
           assertEquals(1256, holder0.value);
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -159,10 +166,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   //cast to float4
   public void testCastFloat4(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
-
+    final BufferAllocator allocator = new TopLevelAllocator();
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -172,23 +179,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       Float4Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float4Vector.class);
       Float4Vector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           Float4Holder holder0 = new Float4Holder();
           a0.get(i, holder0);
           assertEquals(12.56, holder0.value, 0.001);
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -200,9 +209,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastFloat8(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -212,23 +222,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       Float8Vector c0 = exec.getValueVectorById(new SchemaPath("varchar_cast2", ExpressionPosition.UNKNOWN), Float8Vector.class);
       Float8Vector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           Float8Holder holder0 = new Float8Holder();
           a0.get(i, holder0);
           assertEquals(12.56, holder0.value, 0.001);
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -240,9 +252,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastVarChar(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -252,23 +265,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       VarCharVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarCharVector.class);
       VarCharVector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           VarCharHolder holder0 = new VarCharHolder();
           a0.get(i, holder0);
           assertEquals("123", holder0.toString());
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -280,9 +295,11 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastVarBinary(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
+
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -292,23 +309,25 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       VarBinaryVector c0 = exec.getValueVectorById(new SchemaPath("int_lit_cast", ExpressionPosition.UNKNOWN), VarBinaryVector.class);
       VarBinaryVector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           VarBinaryHolder holder0 = new VarBinaryHolder();
           a0.get(i, holder0);
           assertEquals("123", holder0.toString());
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -320,9 +339,10 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastNested(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -332,26 +352,30 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
       IntVector c0 = exec.getValueVectorById(new SchemaPath("add_cast", ExpressionPosition.UNKNOWN),IntVector.class);
       IntVector.Accessor a0;
       a0 = c0.getAccessor();
-      
+
       int count = 0;
-      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){        
+      for(int i = 0; i < c0.getAccessor().getValueCount(); i++){
           IntHolder holder0 = new IntHolder();
           a0.get(i, holder0);
           assertEquals(300, holder0.value);
           ++count;
-        
+
       }
       assertEquals(5, count);
     }
 
+    allocator.close();
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
+
+
     assertTrue(!context.isFailed());
   }
 
@@ -359,9 +383,11 @@ public class TestCastFunctions extends PopUnitTestBase{
   public void testCastNumException(@Injectable final DrillbitContext bitContext,
                             @Injectable UserServer.UserClientConnection connection) throws Throwable{
 
+    final BufferAllocator allocator = new TopLevelAllocator();
+
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
-      bitContext.getAllocator(); result = new TopLevelAllocator();
+      bitContext.getAllocator(); result = allocator;
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
@@ -371,18 +397,20 @@ public class TestCastFunctions extends PopUnitTestBase{
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
     FragmentContext context = new FragmentContext(bitContext, PlanFragment.getDefaultInstance(), connection, registry);
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
-    
+
     while(exec.next()){
     }
 
+    allocator.close();
+
     assertTrue(context.isFailed());
-    
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
 
   }
-  
+
   @Test
   public void testCastFromNullablCol() throws Throwable {
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
@@ -390,43 +418,48 @@ public class TestCastFunctions extends PopUnitTestBase{
     try(Drillbit bit = new Drillbit(CONFIG, serviceSet);
         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator())) {
       bit.run();
-      
+
       client.connect();
       List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
-          Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarCharNull.json"), Charsets.UTF_8).replace("#{TEST_FILE}", "/jsoninput/input1.json"));      
-    
+          Files.toString(FileUtils.getResourceAsFile("/functions/cast/testCastVarCharNull.json"), Charsets.UTF_8).replace("#{TEST_FILE}", "/jsoninput/input1.json"));
+
       QueryResultBatch batch = results.get(0);
 
       RecordBatchLoader batchLoader = new RecordBatchLoader(bit.getContext().getAllocator());
       batchLoader.load(batch.getHeader().getDef(), batch.getData());
-     
+
       Object [][] result = getRunResult(batchLoader);
-      
+
       Object [][] expected = new Object[2][2];
-      
+
       expected[0][0] = new String("2001");
       expected[0][1] = new String("1.2");
-      
+
       expected[1][0] = new String("-2002");
       expected[1][1] = new String("-1.2");
- 
+
       assertEquals(result.length, expected.length);
       assertEquals(result[0].length, expected[0].length);
-      
+
       for (int i = 0; i<result.length; i++ ) {
         for (int j = 0; j<result[0].length; j++) {
           assertEquals(String.format("Column %s at row %s have wrong result",  j, i), result[i][j], expected[i][j]);
         }
-      }     
-    }  
+      }
+      batchLoader.clear();
+      for(QueryResultBatch b : results){
+        b.release();
+      }
+
+    }
   }
 
   private Object[][] getRunResult(VectorAccessible va) {
     int size = 0;
     for (VectorWrapper v : va) {
-      size++;     
-    }   
-  
+      size++;
+    }
+
     Object[][] res = new Object [va.getRecordCount()][size];
     for (int j = 0; j < va.getRecordCount(); j++) {
       int i = 0;
@@ -434,13 +467,13 @@ public class TestCastFunctions extends PopUnitTestBase{
         Object o =  v.getValueVector().getAccessor().getObject(j);
         if (o instanceof byte[]) {
           res[j][i++] =  new String((byte[]) o);
-        } else { 
+        } else {
           res[j][i++] = o;
         }
       }
     }
     return res;
- }  
+ }
 
   @After
   public void tearDown() throws Exception{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
index ac8e5c6..44fd235 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestCastVarCharToBigInt.java
@@ -96,6 +96,11 @@ public class TestCastVarCharToBigInt extends PopUnitTestBase {
                 assertEquals(accessor.getObject(1), 2007L);
                 assertEquals(accessor.getObject(2), 2006L);
             }
+
+            for(QueryResultBatch b : results){
+              b.release();
+            }
+            batchLoader.clear();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
index d008723..b18d996 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestDistributedFragmentRun.java
@@ -40,8 +40,8 @@ import com.google.common.io.Files;
 
 public class TestDistributedFragmentRun extends PopUnitTestBase{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestDistributedFragmentRun.class);
-  
-  @Test 
+
+  @Test
   public void oneBitOneExchangeOneEntryRun() throws Exception{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
@@ -52,10 +52,11 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
       int count = 0;
       for(QueryResultBatch b : results){
         count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(100, count);
     }
-    
+
 
   }
 
@@ -71,6 +72,7 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
       int count = 0;
       for(QueryResultBatch b : results){
         count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(200, count);
     }
@@ -89,6 +91,7 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
             int count = 0;
             for(QueryResultBatch b : results){
                 count += b.getHeader().getRowCount();
+                b.release();
             }
             assertEquals(100, count);
         }
@@ -108,6 +111,7 @@ public class TestDistributedFragmentRun extends PopUnitTestBase{
         int count = 0;
       for(QueryResultBatch b : results){
         count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(200, count);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
index 3e1f7b4..74f1698 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestExtractFunctions.java
@@ -131,6 +131,11 @@ public class TestExtractFunctions extends PopUnitTestBase {
           assertEquals(expectedValues[i][j], vv.getAccessor().get(i));
         }
       }
+
+      for(QueryResultBatch b : results){
+        b.release();
+      }
+      batchLoader.clear();
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
index d20928d..6ed2d25 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestHashToRandomExchange.java
@@ -56,6 +56,7 @@ public class TestHashToRandomExchange extends PopUnitTestBase {
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0)
           count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(200, count);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
index 6289509..c6b254a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestOptiqPlans.java
@@ -51,6 +51,7 @@ import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.data.DataConnectionCreator;
 import org.apache.drill.exec.rpc.user.QueryResultBatch;
 import org.apache.drill.exec.rpc.user.UserServer.UserClientConnection;
+import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.DrillbitContext;
@@ -107,7 +108,7 @@ public class TestOptiqPlans {
     };
     RemoteServiceSet lss = RemoteServiceSet.getLocalServiceSet();
     DrillbitContext bitContext = new DrillbitContext(DrillbitEndpoint.getDefaultInstance(), context, coord, controller, com, cache, workBus);
-    QueryContext qc = new QueryContext(QueryId.getDefaultInstance(), bitContext);
+    QueryContext qc = new QueryContext(new UserSession(null, null, null), QueryId.getDefaultInstance(), bitContext);
     PhysicalPlanReader reader = bitContext.getPlanReader();
     LogicalPlan plan = reader.readLogicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
     PhysicalPlan pp = new BasicOptimizer(DrillConfig.create(), qc).optimize(
@@ -137,7 +138,7 @@ public class TestOptiqPlans {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
+          System.out.println(vw.getValueVector().getField().toExpr());
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);
@@ -165,7 +166,7 @@ public class TestOptiqPlans {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
+          System.out.println(vw.getValueVector().getField().toExpr());
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);
@@ -193,7 +194,7 @@ public class TestOptiqPlans {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper<?> vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
+          System.out.println(vw.getValueVector().getField().toExpr());
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);
@@ -231,7 +232,7 @@ public class TestOptiqPlans {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
+          System.out.println(vw.getValueVector().getField().toExpr());
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);
@@ -269,7 +270,7 @@ public class TestOptiqPlans {
         System.out.println(String.format("Got %d results", b.getHeader().getRowCount()));
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper vw : loader) {
-          System.out.println(vw.getValueVector().getField().getName());
+          System.out.println(vw.getValueVector().getField().toExpr());
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
index 454b15f..842aa8f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestSimpleFragmentRun.java
@@ -52,10 +52,10 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
 
   @Test
   public void runNoExchangeFragment() throws Exception {
-    try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet(); 
-        Drillbit bit = new Drillbit(CONFIG, serviceSet); 
+    try(RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+        Drillbit bit = new Drillbit(CONFIG, serviceSet);
         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());){
-    
+
     // run query.
     bit.run();
     client.connect();
@@ -81,7 +81,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
           } else {
             System.out.print("\t");
           }
-          System.out.print(value.getField().getName());
+          System.out.print(value.getField().toExpr());
           System.out.print("[");
           System.out.print(value.getField().getType().getMinorType());
           System.out.print("]");
@@ -102,6 +102,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
         }
         if(!first) System.out.println();
       }
+      batch.release();
     }
     logger.debug("Received results {}", results);
     assertEquals(recordCount, 200);
@@ -151,7 +152,7 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
           } else {
             System.out.print("\t");
           }
-          System.out.print(v.getField().getName());
+          System.out.print(v.getField().toExpr());
           System.out.print("[");
           System.out.print(v.getField().getType().getMinorType());
           System.out.print("]");
@@ -175,6 +176,8 @@ public class TestSimpleFragmentRun extends PopUnitTestBase {
           }
           if (!first) System.out.println();
         }
+        batchLoader.clear();
+        batch.release();
       }
 
       assertEquals(2, recordCount);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
index 2e16b47..6de4ada 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TestUnionExchange.java
@@ -54,6 +54,7 @@ public class TestUnionExchange extends PopUnitTestBase {
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0)
           count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(150, count);
     }


[09/51] [abbrv] git commit: Support CASE statement in query 1. Pending testcases

Posted by ja...@apache.org.
Support CASE statement in query 1. Pending testcases


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

Branch: refs/heads/master
Commit: 3b9df0de7ae9f6cff0a5726f1ac7929ff5dbcbda
Parents: eae6fa4
Author: vkorukanti <ve...@gmail.com>
Authored: Mon Mar 31 14:30:55 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:08 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/planner/logical/DrillOptiq.java  | 29 +++++++++++++++++---
 1 file changed, 25 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3b9df0de/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 752f31d..405b059 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -24,6 +24,8 @@ import java.util.List;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.FunctionCallFactory;
+import org.apache.drill.common.expression.IfExpression;
+import org.apache.drill.common.expression.IfExpression.IfCondition;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.expression.ValueExpressions;
@@ -120,6 +122,23 @@ public class DrillOptiq {
         case LIKE:
         case SIMILAR:
           return getDrillFunctionFromOptiqCall(call);
+        case CASE:
+          List<LogicalExpression> caseArgs = Lists.newArrayList();
+          for(RexNode r : call.getOperands()){
+            caseArgs.add(r.accept(this));
+          }
+
+          caseArgs = Lists.reverse(caseArgs);
+          // number of arguements are always going to be odd, because
+          // Optiq adds "null" for the missing else expression at the end
+          assert caseArgs.size()%2 == 1;
+          LogicalExpression elseExpression = caseArgs.get(0);
+          for (int i=1; i<caseArgs.size(); i=i+2) {
+            elseExpression = IfExpression.newBuilder()
+              .setElse(elseExpression)
+              .addCondition(new IfCondition(caseArgs.get(i + 1), caseArgs.get(i))).build();
+          }
+          return elseExpression;
         }
         
         if (call.getOperator() == SqlStdOperatorTable.ITEM) {
@@ -227,13 +246,15 @@ public class DrillOptiq {
       case DATE:
         return (ValueExpressions.getDate((GregorianCalendar)literal.getValue()));
       case TIME:
-          return (ValueExpressions.getTime((GregorianCalendar)literal.getValue()));
+        return (ValueExpressions.getTime((GregorianCalendar)literal.getValue()));
       case TIMESTAMP:
-          return (ValueExpressions.getTimeStamp((GregorianCalendar) literal.getValue()));
+        return (ValueExpressions.getTimeStamp((GregorianCalendar) literal.getValue()));
       case INTERVAL_YEAR_MONTH:
-          return (ValueExpressions.getIntervalYear(((BigDecimal) (literal.getValue())).intValue()));
+        return (ValueExpressions.getIntervalYear(((BigDecimal) (literal.getValue())).intValue()));
       case INTERVAL_DAY_TIME:
-          return (ValueExpressions.getIntervalDay(((BigDecimal) (literal.getValue())).longValue()));
+        return (ValueExpressions.getIntervalDay(((BigDecimal) (literal.getValue())).longValue()));
+      case NULL:
+        return NullExpression.INSTANCE;
       default:
         throw new UnsupportedOperationException(String.format("Unable to convert the value of %s and type %s to a Drill constant expression.", literal, literal.getTypeName()));
       }


[35/51] [abbrv] git commit: fix merge join to handle case where right batch has zero records

Posted by ja...@apache.org.
fix merge join to handle case where right batch has zero records


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

Branch: refs/heads/master
Commit: ba123e69da6bdf4977e9db151236c3903ebcc028
Parents: dcc102a
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 22:20:25 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../exec/physical/config/MergeJoinPOP.java      |   1 +
 .../exec/physical/impl/join/JoinStatus.java     |  20 +++-
 .../exec/physical/impl/join/MergeJoinBatch.java | 107 ++++++++++++-------
 .../exec/physical/impl/join/TestMergeJoin.java  |  55 +++++++++-
 .../resources/join/merge_join_empty_batch.json  |   1 +
 5 files changed, 135 insertions(+), 49 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba123e69/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
index a9a0f6d..047bed7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/MergeJoinPOP.java
@@ -62,6 +62,7 @@ public class MergeJoinPOP extends AbstractBase{
     this.right = right;
     this.conditions = conditions;
     this.joinType = joinType;
+    Preconditions.checkArgument(joinType != JoinRelType.FULL, "Full outer join not currently supported");
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba123e69/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
index 5a83b48..baa232e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/JoinStatus.java
@@ -23,6 +23,7 @@ import org.apache.drill.exec.record.RecordBatch;
 import org.apache.drill.exec.record.RecordBatch.IterOutcome;
 import org.apache.drill.exec.record.VectorWrapper;
 import org.apache.drill.exec.record.selection.SelectionVector4;
+import org.eigenbase.rel.JoinRelType;
 
 /**
  * The status of the current join.  Maintained outside the individually compiled join templates so that we can carry status across multiple schemas.
@@ -48,6 +49,8 @@ public final class JoinStatus {
   public MergeJoinBatch outputBatch;
   public SelectionVector4 sv4;
 
+  private final JoinRelType joinType;
+
   public boolean ok = true;
   private boolean initialSet = false;
   private boolean leftRepeating = false;
@@ -57,6 +60,7 @@ public final class JoinStatus {
     this.left = left;
     this.right = right;
     this.outputBatch = output;
+    this.joinType = output.getJoinType();
   }
 
   public final void ensureInitial(){
@@ -204,17 +208,25 @@ public final class JoinStatus {
   public JoinOutcome getOutcome(){
     if (!ok)
       return JoinOutcome.FAILURE;
-    if (lastLeft == IterOutcome.OK && lastRight == IterOutcome.OK)
+    if (bothMatches(IterOutcome.NONE) ||
+            (joinType == JoinRelType.INNER && eitherMatches(IterOutcome.NONE)) ||
+            (joinType == JoinRelType.LEFT && lastLeft == IterOutcome.NONE) ||
+            (joinType == JoinRelType.RIGHT && lastRight == IterOutcome.NONE))
+      return JoinOutcome.NO_MORE_DATA;
+    if (bothMatches(IterOutcome.OK) ||
+            (eitherMatches(IterOutcome.NONE) && eitherMatches(IterOutcome.OK)))
       return JoinOutcome.BATCH_RETURNED;
-    if (eitherMatches(IterOutcome.NONE))
-      return JoinOutcome.NO_MORE_DATA;    
     if (eitherMatches(IterOutcome.OK_NEW_SCHEMA))
       return JoinOutcome.SCHEMA_CHANGED;
     if (eitherMatches(IterOutcome.NOT_YET))
       return JoinOutcome.WAITING;
     return JoinOutcome.FAILURE;
   }
-  
+
+  private boolean bothMatches(IterOutcome outcome){
+    return lastLeft == outcome && lastRight == outcome;
+  }
+
   private boolean eitherMatches(IterOutcome outcome){
     return lastLeft == outcome || lastRight == outcome;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba123e69/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
index 0cb2f7d..bbdfbe5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/join/MergeJoinBatch.java
@@ -25,7 +25,10 @@ import java.util.List;
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.TypedNullConstant;
 import org.apache.drill.common.logical.data.JoinCondition;
+import org.apache.drill.common.types.TypeProtos.MinorType;
+import org.apache.drill.common.types.Types;
 import org.apache.drill.exec.compile.sig.MappingSet;
 import org.apache.drill.exec.exception.ClassTransformationException;
 import org.apache.drill.exec.exception.SchemaChangeException;
@@ -108,10 +111,14 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     }
     this.left = left;
     this.right = right;
+    this.joinType = popConfig.getJoinType();
     this.status = new JoinStatus(left, right, this);
     this.batchBuilder = new MergeJoinBatchBuilder(context, status);
-    this.joinType = popConfig.getJoinType();
-    this.conditions = popConfig.getConditions();   
+    this.conditions = popConfig.getConditions();
+  }
+
+  public JoinRelType getJoinType() {
+    return joinType;
   }
 
   @Override
@@ -319,19 +326,21 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     //////////////////////
     cg.setMappingSet(copyLeftMapping);
     int vectorId = 0;
-    for (VectorWrapper<?> vw : left) {
-      JVar vvIn = cg.declareVectorValueSetupAndMember("incomingLeft",
-                                                      new TypedFieldId(vw.getField().getType(), vectorId));
-      JVar vvOut = cg.declareVectorValueSetupAndMember("outgoing",
-                                                       new TypedFieldId(vw.getField().getType(),vectorId));
-      // todo: check result of copyFromSafe and grow allocation
-      cg.getEvalBlock()._if(vvOut.invoke("copyFromSafe")
-                                   .arg(copyLeftMapping.getValueReadIndex())
-                                   .arg(copyLeftMapping.getValueWriteIndex())
-                                   .arg(vvIn).eq(JExpr.FALSE))
-          ._then()
-          ._return(JExpr.FALSE);
-      ++vectorId;
+    if (status.isLeftPositionAllowed()) {
+      for (VectorWrapper<?> vw : left) {
+        JVar vvIn = cg.declareVectorValueSetupAndMember("incomingLeft",
+                                                        new TypedFieldId(vw.getField().getType(), vectorId));
+        JVar vvOut = cg.declareVectorValueSetupAndMember("outgoing",
+                                                         new TypedFieldId(vw.getField().getType(),vectorId));
+        // todo: check result of copyFromSafe and grow allocation
+        cg.getEvalBlock()._if(vvOut.invoke("copyFromSafe")
+                                     .arg(copyLeftMapping.getValueReadIndex())
+                                     .arg(copyLeftMapping.getValueWriteIndex())
+                                     .arg(vvIn).eq(JExpr.FALSE))
+            ._then()
+            ._return(JExpr.FALSE);
+        ++vectorId;
+      }
     }
     cg.getEvalBlock()._return(JExpr.lit(true));
 
@@ -340,19 +349,21 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     cg.setMappingSet(copyRightMappping);
 
     int rightVectorBase = vectorId;
-    for (VectorWrapper<?> vw : right) {
-      JVar vvIn = cg.declareVectorValueSetupAndMember("incomingRight",
-                                                      new TypedFieldId(vw.getField().getType(), vectorId - rightVectorBase));
-      JVar vvOut = cg.declareVectorValueSetupAndMember("outgoing",
-                                                       new TypedFieldId(vw.getField().getType(),vectorId));
-      // todo: check result of copyFromSafe and grow allocation
-      cg.getEvalBlock()._if(vvOut.invoke("copyFromSafe")
-                                 .arg(copyRightMappping.getValueReadIndex())
-                                 .arg(copyRightMappping.getValueWriteIndex())
-                                 .arg(vvIn).eq(JExpr.FALSE))
-          ._then()
-          ._return(JExpr.FALSE);
-      ++vectorId;
+    if (status.isRightPositionAllowed()) {
+      for (VectorWrapper<?> vw : right) {
+        JVar vvIn = cg.declareVectorValueSetupAndMember("incomingRight",
+                                                        new TypedFieldId(vw.getField().getType(), vectorId - rightVectorBase));
+        JVar vvOut = cg.declareVectorValueSetupAndMember("outgoing",
+                                                         new TypedFieldId(vw.getField().getType(),vectorId));
+        // todo: check result of copyFromSafe and grow allocation
+        cg.getEvalBlock()._if(vvOut.invoke("copyFromSafe")
+                                   .arg(copyRightMappping.getValueReadIndex())
+                                   .arg(copyRightMappping.getValueWriteIndex())
+                                   .arg(vvIn).eq(JExpr.FALSE))
+            ._then()
+            ._return(JExpr.FALSE);
+        ++vectorId;
+      }
     }
     cg.getEvalBlock()._return(JExpr.lit(true));
 
@@ -366,19 +377,25 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
     container.clear();
     
     //estimation of joinBatchSize : max of left/right size, expanded by a factor of 16, which is then bounded by MAX_BATCH_SIZE.
-    int joinBatchSize = Math.min(Math.max(left.getRecordCount() , right.getRecordCount() ) * 16, MAX_BATCH_SIZE);
+    int leftCount = status.isLeftPositionAllowed() ? left.getRecordCount() : 0;
+    int rightCount = status.isRightPositionAllowed() ? right.getRecordCount() : 0;
+    int joinBatchSize = Math.min(Math.max(leftCount, rightCount) * 16, MAX_BATCH_SIZE);
     
-    // add fields from both batches    
-    for (VectorWrapper<?> w : left) {
-      ValueVector outgoingVector = TypeHelper.getNewVector(w.getField(), context.getAllocator());
-      VectorAllocator.getAllocator(outgoingVector, (int) Math.ceil(w.getValueVector().getBufferSize() / left.getRecordCount())).alloc(joinBatchSize);
-      container.add(outgoingVector);
+    // add fields from both batches
+    if (leftCount > 0) {
+      for (VectorWrapper<?> w : left) {
+        ValueVector outgoingVector = TypeHelper.getNewVector(w.getField(), context.getAllocator());
+        VectorAllocator.getAllocator(outgoingVector, (int) Math.ceil(w.getValueVector().getBufferSize() / left.getRecordCount())).alloc(joinBatchSize);
+        container.add(outgoingVector);
+      }
     }
 
-    for (VectorWrapper<?> w : right) {
-      ValueVector outgoingVector = TypeHelper.getNewVector(w.getField(), context.getAllocator());
-      VectorAllocator.getAllocator(outgoingVector, (int) Math.ceil(w.getValueVector().getBufferSize() / right.getRecordCount())).alloc(joinBatchSize);
-      container.add(outgoingVector);
+    if (rightCount > 0) {
+      for (VectorWrapper<?> w : right) {
+        ValueVector outgoingVector = TypeHelper.getNewVector(w.getField(), context.getAllocator());
+        VectorAllocator.getAllocator(outgoingVector, (int) Math.ceil(w.getValueVector().getBufferSize() / right.getRecordCount())).alloc(joinBatchSize);
+        container.add(outgoingVector);
+      }
     }
 
     container.buildSchema(BatchSchema.SelectionVectorMode.NONE);
@@ -395,12 +412,22 @@ public class MergeJoinBatch extends AbstractRecordBatch<MergeJoinPOP> {
       final LogicalExpression rightFieldExpr = condition.getRight();
 
       // materialize value vector readers from join expression
-      final LogicalExpression materializedLeftExpr = ExpressionTreeMaterializer.materialize(leftFieldExpr, left, collector, context.getFunctionRegistry());
+      LogicalExpression materializedLeftExpr;
+      if (status.isLeftPositionAllowed()) {
+        materializedLeftExpr = ExpressionTreeMaterializer.materialize(leftFieldExpr, left, collector, context.getFunctionRegistry());
+      } else {
+        materializedLeftExpr = new TypedNullConstant(Types.optional(MinorType.INT));
+      }
       if (collector.hasErrors())
         throw new ClassTransformationException(String.format(
             "Failure while trying to materialize incoming left field.  Errors:\n %s.", collector.toErrorString()));
 
-      final LogicalExpression materializedRightExpr = ExpressionTreeMaterializer.materialize(rightFieldExpr, right, collector, context.getFunctionRegistry());
+      LogicalExpression materializedRightExpr;
+      if (status.isRightPositionAllowed()) {
+        materializedRightExpr = ExpressionTreeMaterializer.materialize(rightFieldExpr, right, collector, context.getFunctionRegistry());
+      } else {
+        materializedRightExpr = new TypedNullConstant(Types.optional(MinorType.INT));
+      }
       if (collector.hasErrors())
         throw new ClassTransformationException(String.format(
             "Failure while trying to materialize incoming right field.  Errors:\n %s.", collector.toErrorString()));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba123e69/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 8e92181..1623b86 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -53,6 +53,7 @@ import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.vector.ValueVector;
 import org.junit.AfterClass;
+import org.junit.Ignore;
 import org.junit.Test;
 
 import com.beust.jcommander.internal.Lists;
@@ -299,7 +300,29 @@ public class TestMergeJoin extends PopUnitTestBase {
   }
 
   @Test
-  public void testMergeJoinEmptyBatch() throws Exception {
+  public void testMergeJoinInnerEmptyBatch() throws Exception {
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
+
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+              Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
+                      Charsets.UTF_8)
+                      .replace("${JOIN_TYPE}", "INNER"));
+      int count = 0;
+      for(QueryResultBatch b : results) {
+        if (b.getHeader().getRowCount() != 0)
+          count += b.getHeader().getRowCount();
+      }
+      assertEquals(0, count);
+    }
+  }
+
+  @Test
+  public void testMergeJoinLeftEmptyBatch() throws Exception {
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
     try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);      
@@ -309,7 +332,30 @@ public class TestMergeJoin extends PopUnitTestBase {
       client.connect();
       List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
           Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
-              Charsets.UTF_8));
+              Charsets.UTF_8)
+              .replace("${JOIN_TYPE}", "LEFT"));
+      int count = 0;
+      for(QueryResultBatch b : results) {
+        if (b.getHeader().getRowCount() != 0)
+          count += b.getHeader().getRowCount();
+      }
+      assertEquals(50, count);
+    }
+  }
+
+  @Test
+  public void testMergeJoinRightEmptyBatch() throws Exception {
+    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
+
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
+        DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
+
+      bit1.run();
+      client.connect();
+      List<QueryResultBatch> results = client.runQuery(UserProtos.QueryType.PHYSICAL,
+              Files.toString(FileUtils.getResourceAsFile("/join/merge_join_empty_batch.json"),
+                      Charsets.UTF_8)
+                      .replace("${JOIN_TYPE}", "RIGHT"));
       int count = 0;
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0)
@@ -317,9 +363,8 @@ public class TestMergeJoin extends PopUnitTestBase {
       }
       assertEquals(0, count);
     }
-  }  
-  
-  
+  }
+
   @AfterClass
   public static void tearDown() throws Exception{
     // pause to get logger to catch up.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ba123e69/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json b/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
index 549cb83..19254e3 100644
--- a/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
+++ b/exec/java-exec/src/test/resources/join/merge_join_empty_batch.json
@@ -36,6 +36,7 @@
       right: 1,
       left: 2,
       pop: "merge-join",
+      join-type: "${JOIN_TYPE}",
       join-conditions: [ {relationship: "==", left: "blue1", right: "blue"} ]
     },
     {


[18/51] [abbrv] git commit: implement function to cast from long to date, timestamp

Posted by ja...@apache.org.
implement function to cast from long to date, timestamp


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

Branch: refs/heads/master
Commit: 255d528ef93c2fe34c6c5fa4051b711daa4b8271
Parents: 2faf6ef
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 16:20:19 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/expr/fn/impl/CastBigIntDate.java | 46 ++++++++++++++++++++
 .../exec/expr/fn/impl/CastBigIntTimeStamp.java  | 46 ++++++++++++++++++++
 2 files changed, 92 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/255d528e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntDate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntDate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntDate.java
new file mode 100644
index 0000000..e2586d8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntDate.java
@@ -0,0 +1,46 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.DateHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@SuppressWarnings("unused")
+@FunctionTemplate(name = "castDATE", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls= NullHandling.NULL_IF_NULL)
+public class CastBigIntDate implements DrillSimpleFunc {
+
+  @Param
+  BigIntHolder in;
+  @Output
+  DateHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+  }
+
+  @Override
+  public void eval() {
+    out.value = in.value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/255d528e/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntTimeStamp.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntTimeStamp.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntTimeStamp.java
new file mode 100644
index 0000000..ea92f3c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/CastBigIntTimeStamp.java
@@ -0,0 +1,46 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BigIntHolder;
+import org.apache.drill.exec.expr.holders.TimeStampHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+@SuppressWarnings("unused")
+@FunctionTemplate(name = "castTIMESTAMP", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls= NullHandling.NULL_IF_NULL)
+public class CastBigIntTimeStamp implements DrillSimpleFunc {
+
+  @Param
+  BigIntHolder in;
+  @Output
+  TimeStampHolder out;
+
+  @Override
+  public void setup(RecordBatch incoming) {
+  }
+
+  @Override
+  public void eval() {
+    out.value = in.value;
+  }
+}


[40/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testRight.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testRight.json b/exec/java-exec/src/test/resources/functions/string/testRight.json
index db5a5a5..74abcd4 100644
--- a/exec/java-exec/src/test/resources/functions/string/testRight.json
+++ b/exec/java-exec/src/test/resources/functions/string/testRight.json
@@ -23,13 +23,13 @@
             child: 1,
             pop:"project",
             exprs: [
-							{ ref: "col1", expr: "right(\"abcdef\", 2)"},
-							{ ref: "col2", expr: "right(\"abcdef\", 6)"}, 
-              { ref: "col3", expr: "right(\"abcdef\", 7)"},
-              { ref: "col4", expr: "right(\"abcdef\", -2)"},
-              { ref: "col5", expr: "right(\"abcdef\", -5)"},
-              { ref: "col6", expr: "right(\"abcdef\", -6)"},
-              { ref: "col7", expr: "right(\"abcdef\", -7)"}
+							{ ref: "col1", expr: "right('abcdef', 2)"},
+							{ ref: "col2", expr: "right('abcdef', 6)"}, 
+              { ref: "col3", expr: "right('abcdef', 7)"},
+              { ref: "col4", expr: "right('abcdef', -2)"},
+              { ref: "col5", expr: "right('abcdef', -5)"},
+              { ref: "col6", expr: "right('abcdef', -6)"},
+              { ref: "col7", expr: "right('abcdef', -7)"}
      			]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testRpad.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testRpad.json b/exec/java-exec/src/test/resources/functions/string/testRpad.json
index 18f34d9..e41da4b 100644
--- a/exec/java-exec/src/test/resources/functions/string/testRpad.json
+++ b/exec/java-exec/src/test/resources/functions/string/testRpad.json
@@ -23,16 +23,16 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "rpad(\"abcdef\", 0, \"abc\")"},
-						  { ref: "col2", expr: "rpad(\"abcdef\", -3, \"abc\")"},
-						  { ref: "col3", expr: "rpad(\"abcdef\", 6, \"abc\")"},
-						  { ref: "col4", expr: "rpad(\"abcdef\", 2, \"abc\")"},
-						  { ref: "col5", expr: "rpad(\"abcdef\", 2, \"\")"},
-						  { ref: "col7", expr: "rpad(\"abcdef\", 10, \"\")"},
-						  { ref: "col8", expr: "rpad(\"abcdef\", 10, \"A\")"},
-						  { ref: "col9", expr: "rpad(\"abcdef\", 10, \"AB\")"},
-						  { ref: "col10", expr: "rpad(\"abcdef\", 10, \"ABC\")"},
-						  { ref: "col11", expr: "rpad(\"abcdef\", 10, \"ABCDEFGHIJKLMN\")"}
+						  { ref: "col1", expr: "rpad('abcdef', 0, 'abc')"},
+						  { ref: "col2", expr: "rpad('abcdef', -3, 'abc')"},
+						  { ref: "col3", expr: "rpad('abcdef', 6, 'abc')"},
+						  { ref: "col4", expr: "rpad('abcdef', 2, 'abc')"},
+						  { ref: "col5", expr: "rpad('abcdef', 2, '')"},
+						  { ref: "col7", expr: "rpad('abcdef', 10, '')"},
+						  { ref: "col8", expr: "rpad('abcdef', 10, 'A')"},
+						  { ref: "col9", expr: "rpad('abcdef', 10, 'AB')"},
+						  { ref: "col10", expr: "rpad('abcdef', 10, 'ABC')"},
+						  { ref: "col11", expr: "rpad('abcdef', 10, 'ABCDEFGHIJKLMN')"}
 
         ]
         },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testRtrim.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testRtrim.json b/exec/java-exec/src/test/resources/functions/string/testRtrim.json
index 414534a..d3608d2 100644
--- a/exec/java-exec/src/test/resources/functions/string/testRtrim.json
+++ b/exec/java-exec/src/test/resources/functions/string/testRtrim.json
@@ -23,12 +23,12 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "rtrim(\"abcdef\", \"def\")"},
-						  { ref: "col2", expr: "rtrim(\"abcdef\", \"\")"},
-						  { ref: "col3", expr: "rtrim(\"ABdabc\", \"abc\")"},
-						  { ref: "col4", expr: "rtrim(\"abc\", \"abc\")"},
-						  { ref: "col5", expr: "rtrim(\"\", \"abc\")"},
-						  { ref: "col6", expr: "rtrim(\"\", \"\")"}
+						  { ref: "col1", expr: "rtrim('abcdef', 'def')"},
+						  { ref: "col2", expr: "rtrim('abcdef', '')"},
+						  { ref: "col3", expr: "rtrim('ABdabc', 'abc')"},
+						  { ref: "col4", expr: "rtrim('abc', 'abc')"},
+						  { ref: "col5", expr: "rtrim('', 'abc')"},
+						  { ref: "col6", expr: "rtrim('', '')"}
         ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testSimilar.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testSimilar.json b/exec/java-exec/src/test/resources/functions/string/testSimilar.json
index d7f303d..c0972ae 100644
--- a/exec/java-exec/src/test/resources/functions/string/testSimilar.json
+++ b/exec/java-exec/src/test/resources/functions/string/testSimilar.json
@@ -23,10 +23,10 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "issimilar", expr:" similar(\"abc\", \"abc\")" },
-              { ref: "issimilar", expr:" similar(\"abc\", \"a\")" },
-              { ref: "issimilar", expr:" similar(\"abc\", \"%(b|d)%\")" },
-              { ref: "issimilar", expr:" similar(\"abc\", \"(b|c)%\")" }
+              { ref: "issimilar", expr:" similar('abc', 'abc')" },
+              { ref: "issimilar", expr:" similar('abc', 'a')" },
+              { ref: "issimilar", expr:" similar('abc', '%(b|d)%')" },
+              { ref: "issimilar", expr:" similar('abc', '(b|c)%')" }
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testUpper.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testUpper.json b/exec/java-exec/src/test/resources/functions/string/testUpper.json
index 5258367..c155e14 100644
--- a/exec/java-exec/src/test/resources/functions/string/testUpper.json
+++ b/exec/java-exec/src/test/resources/functions/string/testUpper.json
@@ -23,9 +23,9 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "lower", expr: "upper(\"ABcEFgh\")"}, 
-              { ref: "lower", expr: "upper(\"aBc\")"}, 
-              { ref: "lower", expr: "upper(\"\")"}
+              { ref: "lower", expr: "upper('ABcEFgh')"}, 
+              { ref: "lower", expr: "upper('aBc')"}, 
+              { ref: "lower", expr: "upper('')"}
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/join/mj_multi_condition.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/join/mj_multi_condition.json b/exec/java-exec/src/test/resources/join/mj_multi_condition.json
index f27f279..25b1391 100644
--- a/exec/java-exec/src/test/resources/join/mj_multi_condition.json
+++ b/exec/java-exec/src/test/resources/join/mj_multi_condition.json
@@ -60,10 +60,10 @@
     "pop" : "project",
     "@id" : 5,
     "exprs" : [ {
-      "ref" : "output.ps_partkey",
+      "ref" : "ps_partkey",
       "expr" : "ps_partkey"
     }, {
-      "ref" : "output.ps_suppkey",
+      "ref" : "ps_suppkey",
       "expr" : "ps_suppkey"
     } ],
     "child" : 4
@@ -108,16 +108,16 @@
     "pop" : "project",
     "@id" : 10,
     "exprs" : [ {
-      "ref" : "output.l_partkey",
+      "ref" : "l_partkey",
       "expr" : "l_partkey"
     }, {
-      "ref" : "output.l_suppkey",
+      "ref" : "l_suppkey",
       "expr" : "l_suppkey"
     }, {
-      "ref" : "output.ps_partkey",
+      "ref" : "ps_partkey",
       "expr" : "ps_partkey"
     }, {
-      "ref" : "output.ps_suppkey",
+      "ref" : "ps_suppkey",
       "expr" : "ps_suppkey"
     } ],
     "child" : 9

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/logical_string_filter.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/logical_string_filter.json b/exec/java-exec/src/test/resources/logical_string_filter.json
index 28cc5e7..d8875d3 100644
--- a/exec/java-exec/src/test/resources/logical_string_filter.json
+++ b/exec/java-exec/src/test/resources/logical_string_filter.json
@@ -26,7 +26,7 @@
     op : "filter",
     @id : 2,
     input : 1,
-    expr : " (_MAP.R_NAME)  == (\"AFRICA\") "
+    expr : " (_MAP.R_NAME)  == ('AFRICA') "
   }, {
     op : "project",
     @id : 3,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/simple_plan.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/simple_plan.json b/exec/java-exec/src/test/resources/simple_plan.json
index 2457b1f..7ffb504 100644
--- a/exec/java-exec/src/test/resources/simple_plan.json
+++ b/exec/java-exec/src/test/resources/simple_plan.json
@@ -45,11 +45,11 @@
       transforms:[
         {
           ref:"userId",
-          expr:"regex_like('activity.cookie', \"persistent=([^;]*)\")"
+          expr:"regex_like('activity.cookie', 'persistent=([^;]*)')"
         },
         {
           ref:"session",
-          expr:"regex_like('activity.cookie', \"session=([^;]*)\")"
+          expr:"regex_like('activity.cookie', 'session=([^;]*)')"
         }
       ]
     },
@@ -61,11 +61,11 @@
       transforms:[
         {
           ref:"userId",
-          expr:"regex_like('activity.cookie', \"persistent=([^;]*)\")"
+          expr:"regex_like('activity.cookie', 'persistent=([^;]*)')"
         },
         {
           ref:"session",
-          expr:"regex_like('activity.cookie', \"session=([^;]*)\")"
+          expr:"regex_like('activity.cookie', 'session=([^;]*)')"
         }
       ]
     },
@@ -80,7 +80,7 @@
           transforms:[
             {
               ref:"happy",
-              expr:"regex_like('ep2', \"dink\")"
+              expr:"regex_like('ep2', 'dink')"
             }
           ]
         }
@@ -105,7 +105,7 @@
       transforms:[
         {
           ref:"happy",
-          expr:"regex_like('ep2', \"dink\")"
+          expr:"regex_like('ep2', 'dink')"
         }
       ]
     },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a6200cd..bb53650 100644
--- a/pom.xml
+++ b/pom.xml
@@ -253,9 +253,11 @@
         </plugin>
         <plugin>
           <artifactId>maven-surefire-plugin</artifactId>
-          <version>2.15</version>
+          <version>2.17</version>
           <configuration>
-            <argLine>-Xms1g -Xmx1g -XX:MaxDirectMemorySize=6096M </argLine>
+            <argLine>-Xms1g -Xmx2g -XX:MaxDirectMemorySize=6096M </argLine>
+            <forkCount>8</forkCount>
+            <reuseForks>true</reuseForks>
             <additionalClasspathElements>
               <additionalClasspathElement>./sqlparser/src/test/resources/storage-engines.json</additionalClasspathElement>
             </additionalClasspathElements>
@@ -608,7 +610,7 @@
           <dependency>
             <groupId>net.hydromatic</groupId>
             <artifactId>optiq-core</artifactId>
-            <version>0.6-SNAPSHOT</version>
+            <version>0.6</version>
             <exclusions>
               <exclusion>
                 <groupId>org.jgrapht</groupId>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
index f6ef3b1..f91a010 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/BitControl.java
@@ -3131,6 +3131,20 @@ public final class BitControl {
      * </pre>
      */
     long getQueryStartTime();
+
+    // optional .exec.shared.UserCredentials credentials = 15;
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    boolean hasCredentials();
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials();
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder();
   }
   /**
    * Protobuf type {@code exec.bit.control.PlanFragment}
@@ -3267,6 +3281,19 @@ public final class BitControl {
               queryStartTime_ = input.readInt64();
               break;
             }
+            case 122: {
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder subBuilder = null;
+              if (((bitField0_ & 0x00001000) == 0x00001000)) {
+                subBuilder = credentials_.toBuilder();
+              }
+              credentials_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.UserCredentials.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(credentials_);
+                credentials_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00001000;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -3568,6 +3595,28 @@ public final class BitControl {
       return queryStartTime_;
     }
 
+    // optional .exec.shared.UserCredentials credentials = 15;
+    public static final int CREDENTIALS_FIELD_NUMBER = 15;
+    private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_;
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    public boolean hasCredentials() {
+      return ((bitField0_ & 0x00001000) == 0x00001000);
+    }
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
+      return credentials_;
+    }
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
+      return credentials_;
+    }
+
     private void initFields() {
       handle_ = org.apache.drill.exec.proto.ExecProtos.FragmentHandle.getDefaultInstance();
       networkCost_ = 0F;
@@ -3581,6 +3630,7 @@ public final class BitControl {
       memInitial_ = 20000000L;
       memMax_ = 20000000000L;
       queryStartTime_ = 0L;
+      credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -3630,6 +3680,9 @@ public final class BitControl {
       if (((bitField0_ & 0x00000800) == 0x00000800)) {
         output.writeInt64(14, queryStartTime_);
       }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        output.writeMessage(15, credentials_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -3687,6 +3740,10 @@ public final class BitControl {
         size += com.google.protobuf.CodedOutputStream
           .computeInt64Size(14, queryStartTime_);
       }
+      if (((bitField0_ & 0x00001000) == 0x00001000)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(15, credentials_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -3798,6 +3855,7 @@ public final class BitControl {
           getHandleFieldBuilder();
           getAssignmentFieldBuilder();
           getForemanFieldBuilder();
+          getCredentialsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -3842,6 +3900,12 @@ public final class BitControl {
         bitField0_ = (bitField0_ & ~0x00000400);
         queryStartTime_ = 0L;
         bitField0_ = (bitField0_ & ~0x00000800);
+        if (credentialsBuilder_ == null) {
+          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+        } else {
+          credentialsBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00001000);
         return this;
       }
 
@@ -3930,6 +3994,14 @@ public final class BitControl {
           to_bitField0_ |= 0x00000800;
         }
         result.queryStartTime_ = queryStartTime_;
+        if (((from_bitField0_ & 0x00001000) == 0x00001000)) {
+          to_bitField0_ |= 0x00001000;
+        }
+        if (credentialsBuilder_ == null) {
+          result.credentials_ = credentials_;
+        } else {
+          result.credentials_ = credentialsBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -3984,6 +4056,9 @@ public final class BitControl {
         if (other.hasQueryStartTime()) {
           setQueryStartTime(other.getQueryStartTime());
         }
+        if (other.hasCredentials()) {
+          mergeCredentials(other.getCredentials());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -4748,6 +4823,123 @@ public final class BitControl {
         return this;
       }
 
+      // optional .exec.shared.UserCredentials credentials = 15;
+      private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> credentialsBuilder_;
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public boolean hasCredentials() {
+        return ((bitField0_ & 0x00001000) == 0x00001000);
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
+        if (credentialsBuilder_ == null) {
+          return credentials_;
+        } else {
+          return credentialsBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public Builder setCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          credentials_ = value;
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00001000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public Builder setCredentials(
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder builderForValue) {
+        if (credentialsBuilder_ == null) {
+          credentials_ = builderForValue.build();
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00001000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public Builder mergeCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (((bitField0_ & 0x00001000) == 0x00001000) &&
+              credentials_ != org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance()) {
+            credentials_ =
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder(credentials_).mergeFrom(value).buildPartial();
+          } else {
+            credentials_ = value;
+          }
+          onChanged();
+        } else {
+          credentialsBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00001000;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public Builder clearCredentials() {
+        if (credentialsBuilder_ == null) {
+          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+          onChanged();
+        } else {
+          credentialsBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00001000);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder getCredentialsBuilder() {
+        bitField0_ |= 0x00001000;
+        onChanged();
+        return getCredentialsFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
+        if (credentialsBuilder_ != null) {
+          return credentialsBuilder_.getMessageOrBuilder();
+        } else {
+          return credentials_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 15;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> 
+          getCredentialsFieldBuilder() {
+        if (credentialsBuilder_ == null) {
+          credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder>(
+                  credentials_,
+                  getParentForChildren(),
+                  isClean());
+          credentials_ = null;
+        }
+        return credentialsBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.bit.control.PlanFragment)
     }
 
@@ -5480,7 +5672,7 @@ public final class BitControl {
       "shared.DrillPBError\022\024\n\014running_time\030\t \001(" +
       "\003\"k\n\rFragmentState\022\013\n\007SENDING\020\000\022\027\n\023AWAIT" +
       "ING_ALLOCATION\020\001\022\013\n\007RUNNING\020\002\022\014\n\010FINISHE" +
-      "D\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\"\342\002\n\014PlanF" +
+      "D\020\003\022\r\n\tCANCELLED\020\004\022\n\n\006FAILED\020\005\"\225\003\n\014PlanF" +
       "ragment\022(\n\006handle\030\001 \001(\0132\030.exec.bit.Fragm",
       "entHandle\022\024\n\014network_cost\030\004 \001(\002\022\020\n\010cpu_c" +
       "ost\030\005 \001(\002\022\021\n\tdisk_cost\030\006 \001(\002\022\023\n\013memory_c" +
@@ -5489,16 +5681,18 @@ public final class BitControl {
       "leaf_fragment\030\t \001(\010\022\'\n\007foreman\030\013 \001(\0132\026.e" +
       "xec.DrillbitEndpoint\022\035\n\013mem_initial\030\014 \001(" +
       "\003:\01020000000\022\034\n\007mem_max\030\r \001(\003:\0132000000000" +
-      "0\022\030\n\020query_start_time\030\016 \001(\003\"f\n\017WorkQueue" +
-      "Status\022(\n\010endpoint\030\001 \001(\0132\026.exec.Drillbit" +
-      "Endpoint\022\024\n\014queue_length\030\002 \001(\005\022\023\n\013report",
-      "_time\030\003 \001(\003*\332\001\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007" +
-      "\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027REQ_INIATILIZE_F" +
-      "RAGMENT\020\003\022\027\n\023REQ_CANCEL_FRAGMENT\020\006\022\027\n\023RE" +
-      "Q_FRAGMENT_STATUS\020\007\022\022\n\016REQ_BIT_STATUS\020\010\022" +
-      "\030\n\024RESP_FRAGMENT_HANDLE\020\t\022\030\n\024RESP_FRAGME" +
-      "NT_STATUS\020\n\022\023\n\017RESP_BIT_STATUS\020\013B+\n\033org." +
-      "apache.drill.exec.protoB\nBitControlH\001"
+      "0\022\030\n\020query_start_time\030\016 \001(\003\0221\n\013credentia" +
+      "ls\030\017 \001(\0132\034.exec.shared.UserCredentials\"f" +
+      "\n\017WorkQueueStatus\022(\n\010endpoint\030\001 \001(\0132\026.ex",
+      "ec.DrillbitEndpoint\022\024\n\014queue_length\030\002 \001(" +
+      "\005\022\023\n\013report_time\030\003 \001(\003*\332\001\n\007RpcType\022\r\n\tHA" +
+      "NDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE\020\002\022\033\n\027REQ_I" +
+      "NIATILIZE_FRAGMENT\020\003\022\027\n\023REQ_CANCEL_FRAGM" +
+      "ENT\020\006\022\027\n\023REQ_FRAGMENT_STATUS\020\007\022\022\n\016REQ_BI" +
+      "T_STATUS\020\010\022\030\n\024RESP_FRAGMENT_HANDLE\020\t\022\030\n\024" +
+      "RESP_FRAGMENT_STATUS\020\n\022\023\n\017RESP_BIT_STATU" +
+      "S\020\013B+\n\033org.apache.drill.exec.protoB\nBitC" +
+      "ontrolH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5528,7 +5722,7 @@ public final class BitControl {
           internal_static_exec_bit_control_PlanFragment_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_bit_control_PlanFragment_descriptor,
-              new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "Assignment", "LeafFragment", "Foreman", "MemInitial", "MemMax", "QueryStartTime", });
+              new java.lang.String[] { "Handle", "NetworkCost", "CpuCost", "DiskCost", "MemoryCost", "FragmentJson", "Assignment", "LeafFragment", "Foreman", "MemInitial", "MemMax", "QueryStartTime", "Credentials", });
           internal_static_exec_bit_control_WorkQueueStatus_descriptor =
             getDescriptor().getMessageTypes().get(4);
           internal_static_exec_bit_control_WorkQueueStatus_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
index 1b6d242..99b4df8 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserBitShared.java
@@ -116,6 +116,477 @@ public final class UserBitShared {
     // @@protoc_insertion_point(enum_scope:exec.shared.RpcChannel)
   }
 
+  public interface UserCredentialsOrBuilder
+      extends com.google.protobuf.MessageOrBuilder {
+
+    // optional string user_name = 1;
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    boolean hasUserName();
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    java.lang.String getUserName();
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    com.google.protobuf.ByteString
+        getUserNameBytes();
+  }
+  /**
+   * Protobuf type {@code exec.shared.UserCredentials}
+   */
+  public static final class UserCredentials extends
+      com.google.protobuf.GeneratedMessage
+      implements UserCredentialsOrBuilder {
+    // Use UserCredentials.newBuilder() to construct.
+    private UserCredentials(com.google.protobuf.GeneratedMessage.Builder<?> builder) {
+      super(builder);
+      this.unknownFields = builder.getUnknownFields();
+    }
+    private UserCredentials(boolean noInit) { this.unknownFields = com.google.protobuf.UnknownFieldSet.getDefaultInstance(); }
+
+    private static final UserCredentials defaultInstance;
+    public static UserCredentials getDefaultInstance() {
+      return defaultInstance;
+    }
+
+    public UserCredentials getDefaultInstanceForType() {
+      return defaultInstance;
+    }
+
+    private final com.google.protobuf.UnknownFieldSet unknownFields;
+    @java.lang.Override
+    public final com.google.protobuf.UnknownFieldSet
+        getUnknownFields() {
+      return this.unknownFields;
+    }
+    private UserCredentials(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      initFields();
+      int mutable_bitField0_ = 0;
+      com.google.protobuf.UnknownFieldSet.Builder unknownFields =
+          com.google.protobuf.UnknownFieldSet.newBuilder();
+      try {
+        boolean done = false;
+        while (!done) {
+          int tag = input.readTag();
+          switch (tag) {
+            case 0:
+              done = true;
+              break;
+            default: {
+              if (!parseUnknownField(input, unknownFields,
+                                     extensionRegistry, tag)) {
+                done = true;
+              }
+              break;
+            }
+            case 10: {
+              bitField0_ |= 0x00000001;
+              userName_ = input.readBytes();
+              break;
+            }
+          }
+        }
+      } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+        throw e.setUnfinishedMessage(this);
+      } catch (java.io.IOException e) {
+        throw new com.google.protobuf.InvalidProtocolBufferException(
+            e.getMessage()).setUnfinishedMessage(this);
+      } finally {
+        this.unknownFields = unknownFields.build();
+        makeExtensionsImmutable();
+      }
+    }
+    public static final com.google.protobuf.Descriptors.Descriptor
+        getDescriptor() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_UserCredentials_descriptor;
+    }
+
+    protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+        internalGetFieldAccessorTable() {
+      return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_UserCredentials_fieldAccessorTable
+          .ensureFieldAccessorsInitialized(
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.class, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder.class);
+    }
+
+    public static com.google.protobuf.Parser<UserCredentials> PARSER =
+        new com.google.protobuf.AbstractParser<UserCredentials>() {
+      public UserCredentials parsePartialFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws com.google.protobuf.InvalidProtocolBufferException {
+        return new UserCredentials(input, extensionRegistry);
+      }
+    };
+
+    @java.lang.Override
+    public com.google.protobuf.Parser<UserCredentials> getParserForType() {
+      return PARSER;
+    }
+
+    private int bitField0_;
+    // optional string user_name = 1;
+    public static final int USER_NAME_FIELD_NUMBER = 1;
+    private java.lang.Object userName_;
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    public boolean hasUserName() {
+      return ((bitField0_ & 0x00000001) == 0x00000001);
+    }
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    public java.lang.String getUserName() {
+      java.lang.Object ref = userName_;
+      if (ref instanceof java.lang.String) {
+        return (java.lang.String) ref;
+      } else {
+        com.google.protobuf.ByteString bs = 
+            (com.google.protobuf.ByteString) ref;
+        java.lang.String s = bs.toStringUtf8();
+        if (bs.isValidUtf8()) {
+          userName_ = s;
+        }
+        return s;
+      }
+    }
+    /**
+     * <code>optional string user_name = 1;</code>
+     */
+    public com.google.protobuf.ByteString
+        getUserNameBytes() {
+      java.lang.Object ref = userName_;
+      if (ref instanceof java.lang.String) {
+        com.google.protobuf.ByteString b = 
+            com.google.protobuf.ByteString.copyFromUtf8(
+                (java.lang.String) ref);
+        userName_ = b;
+        return b;
+      } else {
+        return (com.google.protobuf.ByteString) ref;
+      }
+    }
+
+    private void initFields() {
+      userName_ = "";
+    }
+    private byte memoizedIsInitialized = -1;
+    public final boolean isInitialized() {
+      byte isInitialized = memoizedIsInitialized;
+      if (isInitialized != -1) return isInitialized == 1;
+
+      memoizedIsInitialized = 1;
+      return true;
+    }
+
+    public void writeTo(com.google.protobuf.CodedOutputStream output)
+                        throws java.io.IOException {
+      getSerializedSize();
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        output.writeBytes(1, getUserNameBytes());
+      }
+      getUnknownFields().writeTo(output);
+    }
+
+    private int memoizedSerializedSize = -1;
+    public int getSerializedSize() {
+      int size = memoizedSerializedSize;
+      if (size != -1) return size;
+
+      size = 0;
+      if (((bitField0_ & 0x00000001) == 0x00000001)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeBytesSize(1, getUserNameBytes());
+      }
+      size += getUnknownFields().getSerializedSize();
+      memoizedSerializedSize = size;
+      return size;
+    }
+
+    private static final long serialVersionUID = 0L;
+    @java.lang.Override
+    protected java.lang.Object writeReplace()
+        throws java.io.ObjectStreamException {
+      return super.writeReplace();
+    }
+
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        com.google.protobuf.ByteString data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        com.google.protobuf.ByteString data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(byte[] data)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        byte[] data,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws com.google.protobuf.InvalidProtocolBufferException {
+      return PARSER.parseFrom(data, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseDelimitedFrom(java.io.InputStream input)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseDelimitedFrom(
+        java.io.InputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseDelimitedFrom(input, extensionRegistry);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        com.google.protobuf.CodedInputStream input)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input);
+    }
+    public static org.apache.drill.exec.proto.UserBitShared.UserCredentials parseFrom(
+        com.google.protobuf.CodedInputStream input,
+        com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+        throws java.io.IOException {
+      return PARSER.parseFrom(input, extensionRegistry);
+    }
+
+    public static Builder newBuilder() { return Builder.create(); }
+    public Builder newBuilderForType() { return newBuilder(); }
+    public static Builder newBuilder(org.apache.drill.exec.proto.UserBitShared.UserCredentials prototype) {
+      return newBuilder().mergeFrom(prototype);
+    }
+    public Builder toBuilder() { return newBuilder(this); }
+
+    @java.lang.Override
+    protected Builder newBuilderForType(
+        com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+      Builder builder = new Builder(parent);
+      return builder;
+    }
+    /**
+     * Protobuf type {@code exec.shared.UserCredentials}
+     */
+    public static final class Builder extends
+        com.google.protobuf.GeneratedMessage.Builder<Builder>
+       implements org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder {
+      public static final com.google.protobuf.Descriptors.Descriptor
+          getDescriptor() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_UserCredentials_descriptor;
+      }
+
+      protected com.google.protobuf.GeneratedMessage.FieldAccessorTable
+          internalGetFieldAccessorTable() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_UserCredentials_fieldAccessorTable
+            .ensureFieldAccessorsInitialized(
+                org.apache.drill.exec.proto.UserBitShared.UserCredentials.class, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder.class);
+      }
+
+      // Construct using org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder()
+      private Builder() {
+        maybeForceBuilderInitialization();
+      }
+
+      private Builder(
+          com.google.protobuf.GeneratedMessage.BuilderParent parent) {
+        super(parent);
+        maybeForceBuilderInitialization();
+      }
+      private void maybeForceBuilderInitialization() {
+        if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+        }
+      }
+      private static Builder create() {
+        return new Builder();
+      }
+
+      public Builder clear() {
+        super.clear();
+        userName_ = "";
+        bitField0_ = (bitField0_ & ~0x00000001);
+        return this;
+      }
+
+      public Builder clone() {
+        return create().mergeFrom(buildPartial());
+      }
+
+      public com.google.protobuf.Descriptors.Descriptor
+          getDescriptorForType() {
+        return org.apache.drill.exec.proto.UserBitShared.internal_static_exec_shared_UserCredentials_descriptor;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials getDefaultInstanceForType() {
+        return org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials build() {
+        org.apache.drill.exec.proto.UserBitShared.UserCredentials result = buildPartial();
+        if (!result.isInitialized()) {
+          throw newUninitializedMessageException(result);
+        }
+        return result;
+      }
+
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials buildPartial() {
+        org.apache.drill.exec.proto.UserBitShared.UserCredentials result = new org.apache.drill.exec.proto.UserBitShared.UserCredentials(this);
+        int from_bitField0_ = bitField0_;
+        int to_bitField0_ = 0;
+        if (((from_bitField0_ & 0x00000001) == 0x00000001)) {
+          to_bitField0_ |= 0x00000001;
+        }
+        result.userName_ = userName_;
+        result.bitField0_ = to_bitField0_;
+        onBuilt();
+        return result;
+      }
+
+      public Builder mergeFrom(com.google.protobuf.Message other) {
+        if (other instanceof org.apache.drill.exec.proto.UserBitShared.UserCredentials) {
+          return mergeFrom((org.apache.drill.exec.proto.UserBitShared.UserCredentials)other);
+        } else {
+          super.mergeFrom(other);
+          return this;
+        }
+      }
+
+      public Builder mergeFrom(org.apache.drill.exec.proto.UserBitShared.UserCredentials other) {
+        if (other == org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance()) return this;
+        if (other.hasUserName()) {
+          bitField0_ |= 0x00000001;
+          userName_ = other.userName_;
+          onChanged();
+        }
+        this.mergeUnknownFields(other.getUnknownFields());
+        return this;
+      }
+
+      public final boolean isInitialized() {
+        return true;
+      }
+
+      public Builder mergeFrom(
+          com.google.protobuf.CodedInputStream input,
+          com.google.protobuf.ExtensionRegistryLite extensionRegistry)
+          throws java.io.IOException {
+        org.apache.drill.exec.proto.UserBitShared.UserCredentials parsedMessage = null;
+        try {
+          parsedMessage = PARSER.parsePartialFrom(input, extensionRegistry);
+        } catch (com.google.protobuf.InvalidProtocolBufferException e) {
+          parsedMessage = (org.apache.drill.exec.proto.UserBitShared.UserCredentials) e.getUnfinishedMessage();
+          throw e;
+        } finally {
+          if (parsedMessage != null) {
+            mergeFrom(parsedMessage);
+          }
+        }
+        return this;
+      }
+      private int bitField0_;
+
+      // optional string user_name = 1;
+      private java.lang.Object userName_ = "";
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public boolean hasUserName() {
+        return ((bitField0_ & 0x00000001) == 0x00000001);
+      }
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public java.lang.String getUserName() {
+        java.lang.Object ref = userName_;
+        if (!(ref instanceof java.lang.String)) {
+          java.lang.String s = ((com.google.protobuf.ByteString) ref)
+              .toStringUtf8();
+          userName_ = s;
+          return s;
+        } else {
+          return (java.lang.String) ref;
+        }
+      }
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public com.google.protobuf.ByteString
+          getUserNameBytes() {
+        java.lang.Object ref = userName_;
+        if (ref instanceof String) {
+          com.google.protobuf.ByteString b = 
+              com.google.protobuf.ByteString.copyFromUtf8(
+                  (java.lang.String) ref);
+          userName_ = b;
+          return b;
+        } else {
+          return (com.google.protobuf.ByteString) ref;
+        }
+      }
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public Builder setUserName(
+          java.lang.String value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        userName_ = value;
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public Builder clearUserName() {
+        bitField0_ = (bitField0_ & ~0x00000001);
+        userName_ = getDefaultInstance().getUserName();
+        onChanged();
+        return this;
+      }
+      /**
+       * <code>optional string user_name = 1;</code>
+       */
+      public Builder setUserNameBytes(
+          com.google.protobuf.ByteString value) {
+        if (value == null) {
+    throw new NullPointerException();
+  }
+  bitField0_ |= 0x00000001;
+        userName_ = value;
+        onChanged();
+        return this;
+      }
+
+      // @@protoc_insertion_point(builder_scope:exec.shared.UserCredentials)
+    }
+
+    static {
+      defaultInstance = new UserCredentials(true);
+      defaultInstance.initFields();
+    }
+
+    // @@protoc_insertion_point(class_scope:exec.shared.UserCredentials)
+  }
+
   public interface QueryIdOrBuilder
       extends com.google.protobuf.MessageOrBuilder {
 
@@ -4719,6 +5190,11 @@ public final class UserBitShared {
   }
 
   private static com.google.protobuf.Descriptors.Descriptor
+    internal_static_exec_shared_UserCredentials_descriptor;
+  private static
+    com.google.protobuf.GeneratedMessage.FieldAccessorTable
+      internal_static_exec_shared_UserCredentials_fieldAccessorTable;
+  private static com.google.protobuf.Descriptors.Descriptor
     internal_static_exec_shared_QueryId_descriptor;
   private static
     com.google.protobuf.GeneratedMessage.FieldAccessorTable
@@ -4753,57 +5229,64 @@ public final class UserBitShared {
   static {
     java.lang.String[] descriptorData = {
       "\n\023UserBitShared.proto\022\013exec.shared\032\022Coor" +
-      "dination.proto\032\017SchemaDef.proto\"\'\n\007Query" +
-      "Id\022\r\n\005part1\030\001 \001(\020\022\r\n\005part2\030\002 \001(\020\"\241\001\n\014Dri" +
-      "llPBError\022\020\n\010error_id\030\001 \001(\t\022(\n\010endpoint\030" +
-      "\002 \001(\0132\026.exec.DrillbitEndpoint\022\022\n\nerror_t" +
-      "ype\030\003 \001(\005\022\017\n\007message\030\004 \001(\t\0220\n\rparsing_er" +
-      "ror\030\005 \003(\0132\031.exec.shared.ParsingError\"\\\n\014" +
-      "ParsingError\022\024\n\014start_column\030\002 \001(\005\022\021\n\tst" +
-      "art_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001(\005\022\017\n\007end" +
-      "_row\030\005 \001(\005\"p\n\016RecordBatchDef\022)\n\005field\030\001 ",
-      "\003(\0132\032.exec.shared.FieldMetadata\022\024\n\014recor" +
-      "d_count\030\002 \001(\005\022\035\n\025is_selection_vector_2\030\003" +
-      " \001(\010\"\261\001\n\rFieldMetadata\022\033\n\003def\030\001 \001(\0132\016.ex" +
-      "ec.FieldDef\022\023\n\013value_count\030\002 \001(\005\022\027\n\017var_" +
-      "byte_length\030\003 \001(\005\022\023\n\013group_count\030\004 \001(\005\022\025" +
-      "\n\rbuffer_length\030\005 \001(\005\022)\n\005child\030\006 \003(\0132\032.e" +
-      "xec.shared.FieldMetadata*5\n\nRpcChannel\022\017" +
-      "\n\013BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002B" +
-      ".\n\033org.apache.drill.exec.protoB\rUserBitS" +
-      "haredH\001"
+      "dination.proto\032\017SchemaDef.proto\"$\n\017UserC" +
+      "redentials\022\021\n\tuser_name\030\001 \001(\t\"\'\n\007QueryId" +
+      "\022\r\n\005part1\030\001 \001(\020\022\r\n\005part2\030\002 \001(\020\"\241\001\n\014Drill" +
+      "PBError\022\020\n\010error_id\030\001 \001(\t\022(\n\010endpoint\030\002 " +
+      "\001(\0132\026.exec.DrillbitEndpoint\022\022\n\nerror_typ" +
+      "e\030\003 \001(\005\022\017\n\007message\030\004 \001(\t\0220\n\rparsing_erro" +
+      "r\030\005 \003(\0132\031.exec.shared.ParsingError\"\\\n\014Pa" +
+      "rsingError\022\024\n\014start_column\030\002 \001(\005\022\021\n\tstar" +
+      "t_row\030\003 \001(\005\022\022\n\nend_column\030\004 \001(\005\022\017\n\007end_r",
+      "ow\030\005 \001(\005\"p\n\016RecordBatchDef\022)\n\005field\030\001 \003(" +
+      "\0132\032.exec.shared.FieldMetadata\022\024\n\014record_" +
+      "count\030\002 \001(\005\022\035\n\025is_selection_vector_2\030\003 \001" +
+      "(\010\"\261\001\n\rFieldMetadata\022\033\n\003def\030\001 \001(\0132\016.exec" +
+      ".FieldDef\022\023\n\013value_count\030\002 \001(\005\022\027\n\017var_by" +
+      "te_length\030\003 \001(\005\022\023\n\013group_count\030\004 \001(\005\022\025\n\r" +
+      "buffer_length\030\005 \001(\005\022)\n\005child\030\006 \003(\0132\032.exe" +
+      "c.shared.FieldMetadata*5\n\nRpcChannel\022\017\n\013" +
+      "BIT_CONTROL\020\000\022\014\n\010BIT_DATA\020\001\022\010\n\004USER\020\002B.\n" +
+      "\033org.apache.drill.exec.protoB\rUserBitSha",
+      "redH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
         public com.google.protobuf.ExtensionRegistry assignDescriptors(
             com.google.protobuf.Descriptors.FileDescriptor root) {
           descriptor = root;
-          internal_static_exec_shared_QueryId_descriptor =
+          internal_static_exec_shared_UserCredentials_descriptor =
             getDescriptor().getMessageTypes().get(0);
+          internal_static_exec_shared_UserCredentials_fieldAccessorTable = new
+            com.google.protobuf.GeneratedMessage.FieldAccessorTable(
+              internal_static_exec_shared_UserCredentials_descriptor,
+              new java.lang.String[] { "UserName", });
+          internal_static_exec_shared_QueryId_descriptor =
+            getDescriptor().getMessageTypes().get(1);
           internal_static_exec_shared_QueryId_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_QueryId_descriptor,
               new java.lang.String[] { "Part1", "Part2", });
           internal_static_exec_shared_DrillPBError_descriptor =
-            getDescriptor().getMessageTypes().get(1);
+            getDescriptor().getMessageTypes().get(2);
           internal_static_exec_shared_DrillPBError_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_DrillPBError_descriptor,
               new java.lang.String[] { "ErrorId", "Endpoint", "ErrorType", "Message", "ParsingError", });
           internal_static_exec_shared_ParsingError_descriptor =
-            getDescriptor().getMessageTypes().get(2);
+            getDescriptor().getMessageTypes().get(3);
           internal_static_exec_shared_ParsingError_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_ParsingError_descriptor,
               new java.lang.String[] { "StartColumn", "StartRow", "EndColumn", "EndRow", });
           internal_static_exec_shared_RecordBatchDef_descriptor =
-            getDescriptor().getMessageTypes().get(3);
+            getDescriptor().getMessageTypes().get(4);
           internal_static_exec_shared_RecordBatchDef_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_RecordBatchDef_descriptor,
               new java.lang.String[] { "Field", "RecordCount", "IsSelectionVector2", });
           internal_static_exec_shared_FieldMetadata_descriptor =
-            getDescriptor().getMessageTypes().get(4);
+            getDescriptor().getMessageTypes().get(5);
           internal_static_exec_shared_FieldMetadata_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_shared_FieldMetadata_descriptor,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
----------------------------------------------------------------------
diff --git a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
index b8efa84..f8729a9 100644
--- a/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
+++ b/protocol/src/main/java/org/apache/drill/exec/proto/UserProtos.java
@@ -403,6 +403,20 @@ public final class UserProtos {
      * <code>optional int32 rpc_version = 3;</code>
      */
     int getRpcVersion();
+
+    // optional .exec.shared.UserCredentials credentials = 4;
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    boolean hasCredentials();
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials();
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder();
   }
   /**
    * Protobuf type {@code exec.user.UserToBitHandshake}
@@ -476,6 +490,19 @@ public final class UserProtos {
               rpcVersion_ = input.readInt32();
               break;
             }
+            case 34: {
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder subBuilder = null;
+              if (((bitField0_ & 0x00000008) == 0x00000008)) {
+                subBuilder = credentials_.toBuilder();
+              }
+              credentials_ = input.readMessage(org.apache.drill.exec.proto.UserBitShared.UserCredentials.PARSER, extensionRegistry);
+              if (subBuilder != null) {
+                subBuilder.mergeFrom(credentials_);
+                credentials_ = subBuilder.buildPartial();
+              }
+              bitField0_ |= 0x00000008;
+              break;
+            }
           }
         }
       } catch (com.google.protobuf.InvalidProtocolBufferException e) {
@@ -564,10 +591,33 @@ public final class UserProtos {
       return rpcVersion_;
     }
 
+    // optional .exec.shared.UserCredentials credentials = 4;
+    public static final int CREDENTIALS_FIELD_NUMBER = 4;
+    private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_;
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    public boolean hasCredentials() {
+      return ((bitField0_ & 0x00000008) == 0x00000008);
+    }
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
+      return credentials_;
+    }
+    /**
+     * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+     */
+    public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
+      return credentials_;
+    }
+
     private void initFields() {
       channel_ = org.apache.drill.exec.proto.UserBitShared.RpcChannel.USER;
       supportListening_ = false;
       rpcVersion_ = 0;
+      credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
     }
     private byte memoizedIsInitialized = -1;
     public final boolean isInitialized() {
@@ -590,6 +640,9 @@ public final class UserProtos {
       if (((bitField0_ & 0x00000004) == 0x00000004)) {
         output.writeInt32(3, rpcVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        output.writeMessage(4, credentials_);
+      }
       getUnknownFields().writeTo(output);
     }
 
@@ -611,6 +664,10 @@ public final class UserProtos {
         size += com.google.protobuf.CodedOutputStream
           .computeInt32Size(3, rpcVersion_);
       }
+      if (((bitField0_ & 0x00000008) == 0x00000008)) {
+        size += com.google.protobuf.CodedOutputStream
+          .computeMessageSize(4, credentials_);
+      }
       size += getUnknownFields().getSerializedSize();
       memoizedSerializedSize = size;
       return size;
@@ -719,6 +776,7 @@ public final class UserProtos {
       }
       private void maybeForceBuilderInitialization() {
         if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) {
+          getCredentialsFieldBuilder();
         }
       }
       private static Builder create() {
@@ -733,6 +791,12 @@ public final class UserProtos {
         bitField0_ = (bitField0_ & ~0x00000002);
         rpcVersion_ = 0;
         bitField0_ = (bitField0_ & ~0x00000004);
+        if (credentialsBuilder_ == null) {
+          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+        } else {
+          credentialsBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
         return this;
       }
 
@@ -773,6 +837,14 @@ public final class UserProtos {
           to_bitField0_ |= 0x00000004;
         }
         result.rpcVersion_ = rpcVersion_;
+        if (((from_bitField0_ & 0x00000008) == 0x00000008)) {
+          to_bitField0_ |= 0x00000008;
+        }
+        if (credentialsBuilder_ == null) {
+          result.credentials_ = credentials_;
+        } else {
+          result.credentials_ = credentialsBuilder_.build();
+        }
         result.bitField0_ = to_bitField0_;
         onBuilt();
         return result;
@@ -798,6 +870,9 @@ public final class UserProtos {
         if (other.hasRpcVersion()) {
           setRpcVersion(other.getRpcVersion());
         }
+        if (other.hasCredentials()) {
+          mergeCredentials(other.getCredentials());
+        }
         this.mergeUnknownFields(other.getUnknownFields());
         return this;
       }
@@ -927,6 +1002,123 @@ public final class UserProtos {
         return this;
       }
 
+      // optional .exec.shared.UserCredentials credentials = 4;
+      private org.apache.drill.exec.proto.UserBitShared.UserCredentials credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> credentialsBuilder_;
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public boolean hasCredentials() {
+        return ((bitField0_ & 0x00000008) == 0x00000008);
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials getCredentials() {
+        if (credentialsBuilder_ == null) {
+          return credentials_;
+        } else {
+          return credentialsBuilder_.getMessage();
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public Builder setCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (value == null) {
+            throw new NullPointerException();
+          }
+          credentials_ = value;
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public Builder setCredentials(
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder builderForValue) {
+        if (credentialsBuilder_ == null) {
+          credentials_ = builderForValue.build();
+          onChanged();
+        } else {
+          credentialsBuilder_.setMessage(builderForValue.build());
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public Builder mergeCredentials(org.apache.drill.exec.proto.UserBitShared.UserCredentials value) {
+        if (credentialsBuilder_ == null) {
+          if (((bitField0_ & 0x00000008) == 0x00000008) &&
+              credentials_ != org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance()) {
+            credentials_ =
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials.newBuilder(credentials_).mergeFrom(value).buildPartial();
+          } else {
+            credentials_ = value;
+          }
+          onChanged();
+        } else {
+          credentialsBuilder_.mergeFrom(value);
+        }
+        bitField0_ |= 0x00000008;
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public Builder clearCredentials() {
+        if (credentialsBuilder_ == null) {
+          credentials_ = org.apache.drill.exec.proto.UserBitShared.UserCredentials.getDefaultInstance();
+          onChanged();
+        } else {
+          credentialsBuilder_.clear();
+        }
+        bitField0_ = (bitField0_ & ~0x00000008);
+        return this;
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder getCredentialsBuilder() {
+        bitField0_ |= 0x00000008;
+        onChanged();
+        return getCredentialsFieldBuilder().getBuilder();
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      public org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder getCredentialsOrBuilder() {
+        if (credentialsBuilder_ != null) {
+          return credentialsBuilder_.getMessageOrBuilder();
+        } else {
+          return credentials_;
+        }
+      }
+      /**
+       * <code>optional .exec.shared.UserCredentials credentials = 4;</code>
+       */
+      private com.google.protobuf.SingleFieldBuilder<
+          org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder> 
+          getCredentialsFieldBuilder() {
+        if (credentialsBuilder_ == null) {
+          credentialsBuilder_ = new com.google.protobuf.SingleFieldBuilder<
+              org.apache.drill.exec.proto.UserBitShared.UserCredentials, org.apache.drill.exec.proto.UserBitShared.UserCredentials.Builder, org.apache.drill.exec.proto.UserBitShared.UserCredentialsOrBuilder>(
+                  credentials_,
+                  getParentForChildren(),
+                  isClean());
+          credentials_ = null;
+        }
+        return credentialsBuilder_;
+      }
+
       // @@protoc_insertion_point(builder_scope:exec.user.UserToBitHandshake)
     }
 
@@ -5207,37 +5399,39 @@ public final class UserProtos {
   static {
     java.lang.String[] descriptorData = {
       "\n\nUser.proto\022\texec.user\032\017SchemaDef.proto" +
-      "\032\023UserBitShared.proto\"t\n\022UserToBitHandsh" +
-      "ake\022.\n\007channel\030\001 \001(\0162\027.exec.shared.RpcCh" +
-      "annel:\004USER\022\031\n\021support_listening\030\002 \001(\010\022\023" +
-      "\n\013rpc_version\030\003 \001(\005\"S\n\016RequestResults\022&\n" +
-      "\010query_id\030\001 \001(\0132\024.exec.shared.QueryId\022\031\n" +
-      "\021maximum_responses\030\002 \001(\005\"o\n\010RunQuery\0221\n\014" +
-      "results_mode\030\001 \001(\0162\033.exec.user.QueryResu" +
-      "ltsMode\022\"\n\004type\030\002 \001(\0162\024.exec.user.QueryT" +
-      "ype\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUserHandshake\022",
-      "\023\n\013rpc_version\030\002 \001(\005\"7\n\nNodeStatus\022\017\n\007no" +
-      "de_id\030\001 \001(\005\022\030\n\020memory_footprint\030\002 \001(\003\"\331\003" +
-      "\n\013QueryResult\0226\n\013query_state\030\001 \001(\0162!.exe" +
-      "c.user.QueryResult.QueryState\022&\n\010query_i" +
-      "d\030\002 \001(\0132\024.exec.shared.QueryId\022\025\n\ris_last" +
-      "_chunk\030\003 \001(\010\022\021\n\trow_count\030\004 \001(\005\022\024\n\014recor" +
-      "ds_scan\030\005 \001(\003\022\025\n\rrecords_error\030\006 \001(\003\022\027\n\017" +
-      "submission_time\030\007 \001(\003\022*\n\013node_status\030\010 \003" +
-      "(\0132\025.exec.user.NodeStatus\022(\n\005error\030\t \003(\013" +
-      "2\031.exec.shared.DrillPBError\022(\n\003def\030\n \001(\013",
-      "2\033.exec.shared.RecordBatchDef\022\026\n\016schema_" +
-      "changed\030\013 \001(\010\"b\n\nQueryState\022\013\n\007PENDING\020\000" +
-      "\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002\022\014\n\010CANCELED" +
-      "\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QUERY\020\005*\270\001\n\007Rp" +
-      "cType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020\001\022\013\n\007GOODBYE" +
-      "\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_QUERY\020\004\022\023\n\017R" +
-      "EQUEST_RESULTS\020\005\022\020\n\014QUERY_RESULT\020\006\022\020\n\014QU" +
-      "ERY_HANDLE\020\007\022\026\n\022REQ_META_FUNCTIONS\020\010\022\026\n\022" +
-      "RESP_FUNCTION_LIST\020\t*/\n\tQueryType\022\007\n\003SQL" +
-      "\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020\003*#\n\020QueryRe",
-      "sultsMode\022\017\n\013STREAM_FULL\020\001B+\n\033org.apache" +
-      ".drill.exec.protoB\nUserProtosH\001"
+      "\032\023UserBitShared.proto\"\247\001\n\022UserToBitHands" +
+      "hake\022.\n\007channel\030\001 \001(\0162\027.exec.shared.RpcC" +
+      "hannel:\004USER\022\031\n\021support_listening\030\002 \001(\010\022" +
+      "\023\n\013rpc_version\030\003 \001(\005\0221\n\013credentials\030\004 \001(" +
+      "\0132\034.exec.shared.UserCredentials\"S\n\016Reque" +
+      "stResults\022&\n\010query_id\030\001 \001(\0132\024.exec.share" +
+      "d.QueryId\022\031\n\021maximum_responses\030\002 \001(\005\"o\n\010" +
+      "RunQuery\0221\n\014results_mode\030\001 \001(\0162\033.exec.us" +
+      "er.QueryResultsMode\022\"\n\004type\030\002 \001(\0162\024.exec",
+      ".user.QueryType\022\014\n\004plan\030\003 \001(\t\")\n\022BitToUs" +
+      "erHandshake\022\023\n\013rpc_version\030\002 \001(\005\"7\n\nNode" +
+      "Status\022\017\n\007node_id\030\001 \001(\005\022\030\n\020memory_footpr" +
+      "int\030\002 \001(\003\"\331\003\n\013QueryResult\0226\n\013query_state" +
+      "\030\001 \001(\0162!.exec.user.QueryResult.QueryStat" +
+      "e\022&\n\010query_id\030\002 \001(\0132\024.exec.shared.QueryI" +
+      "d\022\025\n\ris_last_chunk\030\003 \001(\010\022\021\n\trow_count\030\004 " +
+      "\001(\005\022\024\n\014records_scan\030\005 \001(\003\022\025\n\rrecords_err" +
+      "or\030\006 \001(\003\022\027\n\017submission_time\030\007 \001(\003\022*\n\013nod" +
+      "e_status\030\010 \003(\0132\025.exec.user.NodeStatus\022(\n",
+      "\005error\030\t \003(\0132\031.exec.shared.DrillPBError\022" +
+      "(\n\003def\030\n \001(\0132\033.exec.shared.RecordBatchDe" +
+      "f\022\026\n\016schema_changed\030\013 \001(\010\"b\n\nQueryState\022" +
+      "\013\n\007PENDING\020\000\022\013\n\007RUNNING\020\001\022\r\n\tCOMPLETED\020\002" +
+      "\022\014\n\010CANCELED\020\003\022\n\n\006FAILED\020\004\022\021\n\rUNKNOWN_QU" +
+      "ERY\020\005*\270\001\n\007RpcType\022\r\n\tHANDSHAKE\020\000\022\007\n\003ACK\020" +
+      "\001\022\013\n\007GOODBYE\020\002\022\r\n\tRUN_QUERY\020\003\022\020\n\014CANCEL_" +
+      "QUERY\020\004\022\023\n\017REQUEST_RESULTS\020\005\022\020\n\014QUERY_RE" +
+      "SULT\020\006\022\020\n\014QUERY_HANDLE\020\007\022\026\n\022REQ_META_FUN" +
+      "CTIONS\020\010\022\026\n\022RESP_FUNCTION_LIST\020\t*/\n\tQuer",
+      "yType\022\007\n\003SQL\020\001\022\013\n\007LOGICAL\020\002\022\014\n\010PHYSICAL\020" +
+      "\003*#\n\020QueryResultsMode\022\017\n\013STREAM_FULL\020\001B+" +
+      "\n\033org.apache.drill.exec.protoB\nUserProto" +
+      "sH\001"
     };
     com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner =
       new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() {
@@ -5249,7 +5443,7 @@ public final class UserProtos {
           internal_static_exec_user_UserToBitHandshake_fieldAccessorTable = new
             com.google.protobuf.GeneratedMessage.FieldAccessorTable(
               internal_static_exec_user_UserToBitHandshake_descriptor,
-              new java.lang.String[] { "Channel", "SupportListening", "RpcVersion", });
+              new java.lang.String[] { "Channel", "SupportListening", "RpcVersion", "Credentials", });
           internal_static_exec_user_RequestResults_descriptor =
             getDescriptor().getMessageTypes().get(1);
           internal_static_exec_user_RequestResults_fieldAccessorTable = new

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/protobuf/BitControl.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/BitControl.proto b/protocol/src/main/protobuf/BitControl.proto
index 95ec7e0..25cae11 100644
--- a/protocol/src/main/protobuf/BitControl.proto
+++ b/protocol/src/main/protobuf/BitControl.proto
@@ -75,6 +75,7 @@ message PlanFragment {
   optional int64 mem_initial = 12 [default = 20000000]; // 20 megs
   optional int64 mem_max = 13 [default = 20000000000]; // 20 gigs
   optional int64 query_start_time = 14; // start time of query in milliseconds
+  optional exec.shared.UserCredentials credentials = 15; 
 }
 
 message WorkQueueStatus{

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/protobuf/User.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/User.proto b/protocol/src/main/protobuf/User.proto
index 3bbe039..78447f7 100644
--- a/protocol/src/main/protobuf/User.proto
+++ b/protocol/src/main/protobuf/User.proto
@@ -35,8 +35,10 @@ message UserToBitHandshake {
   optional exec.shared.RpcChannel channel = 1 [default = USER];
   optional bool support_listening = 2;
   optional int32 rpc_version = 3;
+  optional exec.shared.UserCredentials credentials = 4;
 }
 
+
 message RequestResults {
   optional exec.shared.QueryId query_id = 1;
   optional int32 maximum_responses = 2;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/protocol/src/main/protobuf/UserBitShared.proto
----------------------------------------------------------------------
diff --git a/protocol/src/main/protobuf/UserBitShared.proto b/protocol/src/main/protobuf/UserBitShared.proto
index d75a7f9..0eaa163 100644
--- a/protocol/src/main/protobuf/UserBitShared.proto
+++ b/protocol/src/main/protobuf/UserBitShared.proto
@@ -13,6 +13,10 @@ enum RpcChannel {
   USER = 2;
 }
 
+message UserCredentials {
+  optional string user_name = 1;
+}
+
 message QueryId {
   optional sfixed64 part1 = 1;
   optional sfixed64 part2 = 2;
@@ -46,4 +50,4 @@ message FieldMetadata {
   optional int32 group_count = 4; // number of groups.  (number of repeated records)
   optional int32 buffer_length = 5;
   repeated FieldMetadata child = 6;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/sqlparser/pom.xml
----------------------------------------------------------------------
diff --git a/sqlparser/pom.xml b/sqlparser/pom.xml
index c32eb0b..3ba557c 100644
--- a/sqlparser/pom.xml
+++ b/sqlparser/pom.xml
@@ -79,6 +79,14 @@
   <build>
     <plugins>
       <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-surefire-plugin</artifactId>
+        <configuration>
+            <forkCount>1</forkCount>
+            <reuseForks>true</reuseForks>
+        </configuration>
+      </plugin>
+      <plugin>
         <groupId>org.apache.rat</groupId>
         <artifactId>apache-rat-plugin</artifactId>
         <inherited>true</inherited>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
index 5119212..25d6226 100644
--- a/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
+++ b/sqlparser/src/main/java/org/apache/drill/jdbc/DrillColumnMetaDataList.java
@@ -33,17 +33,17 @@ public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillColumnMetaDataList.class);
 
   private ColumnMetaData[] columns = new ColumnMetaData[0];
-  
+
   @Override
   public int size() {
     return columns.length;
   }
-  
+
   @Override
   public ColumnMetaData get(int index) {
     return columns[index];
   }
-  
+
   public void updateColumnMetaData(String catalogName, String schemaName, String tableName, BatchSchema schema){
 
     columns = new ColumnMetaData[schema.getFieldCount()];
@@ -56,11 +56,11 @@ public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
           true, // caseSensitive
           false, // searchable
           false, // currency
-          f.getDataMode() == DataMode.OPTIONAL ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls, //nullability 
+          f.getDataMode() == DataMode.OPTIONAL ? ResultSetMetaData.columnNullable : ResultSetMetaData.columnNoNulls, //nullability
           !Types.isUnSigned(t), // signed
           10, // display size.
-          f.getName(), // label
-          f.getName(), // columnname
+          f.getAsSchemaPath().getRootSegment().getPath(), // label
+          f.getAsSchemaPath().getRootSegment().getPath(), // columnname
           schemaName, // schemaname
           t.hasPrecision() ? t.getPrecision() : 0, // precision
           t.hasScale() ? t.getScale() : 0, // scale
@@ -77,5 +77,5 @@ public class DrillColumnMetaDataList extends BasicList<ColumnMetaData>{
       columns[i] =col;
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
index e6027fc..e2666f8 100644
--- a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
+++ b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcQuery.java
@@ -39,7 +39,7 @@ import com.google.common.base.Stopwatch;
 public class TestJdbcQuery {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestJdbcQuery.class);
 
-  
+
   // Set a timeout unless we're debugging.
   @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(200000000);
 
@@ -47,14 +47,14 @@ public class TestJdbcQuery {
   static{
     Driver.load();
     WORKING_PATH = Paths.get("").toAbsolutePath().toString();
-    
+
   }
-  
+
   @BeforeClass
   public static void generateHive() throws Exception{
     new HiveTestDataGenerator().generateTestData();
   }
-  
+
   @Test
   @Ignore
   public void testHiveRead() throws Exception{
@@ -73,6 +73,7 @@ public class TestJdbcQuery {
     testQuery("select * from cp.`employee.json`");
   }
 
+
   @Test
   public void testInfoSchema() throws Exception{
 //    testQuery("select * from INFORMATION_SCHEMA.SCHEMATA");
@@ -81,19 +82,19 @@ public class TestJdbcQuery {
 //    testQuery("select * from INFORMATION_SCHEMA.TABLES");
 //    testQuery("select * from INFORMATION_SCHEMA.COLUMNS");
   }
-  
-  @Test 
+
+  @Test
   public void testCast() throws Exception{
-    testQuery(String.format("select R_REGIONKEY, cast(R_NAME as varchar(15)) as region, cast(R_COMMENT as varchar(255)) as comment from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));    
+    testQuery(String.format("select R_REGIONKEY, cast(R_NAME as varchar(15)) as region, cast(R_COMMENT as varchar(255)) as comment from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
-  @Test 
+  @Test
   @Ignore
   public void testWorkspace() throws Exception{
     testQuery(String.format("select * from dfs.home.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
-  @Test 
+  @Test
   @Ignore
   public void testWildcard() throws Exception{
     testQuery(String.format("select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
@@ -109,25 +110,24 @@ public class TestJdbcQuery {
     testQuery(String.format("select cast('test literal' as VARCHAR) from INFORMATION_SCHEMA.TABLES LIMIT 1"));
   }
 
-  @Test 
+  @Test
   @Ignore
   public void testLogicalExplain() throws Exception{
     testQuery(String.format("EXPLAIN PLAN WITHOUT IMPLEMENTATION FOR select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
-  @Test 
+  @Test
   @Ignore
   public void testPhysicalExplain() throws Exception{
     testQuery(String.format("EXPLAIN PLAN FOR select * from dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
-  
-  @Test 
+
+  @Test
   @Ignore
   public void checkUnknownColumn() throws Exception{
     testQuery(String.format("SELECT unknownColumn FROM dfs.`%s/../sample-data/region.parquet`", WORKING_PATH));
   }
 
-  
   private void testQuery(String sql) throws Exception{
     boolean success = false;
     try (Connection c = DriverManager.getConnection("jdbc:drill:zk=local", null);) {


[36/51] [abbrv] git commit: lower quantity threshold in tpch query 18 so that more than zero records are returned

Posted by ja...@apache.org.
lower quantity threshold in tpch query 18 so that more than zero records are returned


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

Branch: refs/heads/master
Commit: 63b0da38b5db8e5a6d00e5552b009fd2c543bd75
Parents: ba123e6
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 22:21:25 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 exec/java-exec/src/test/resources/queries/tpch/18.sql | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/63b0da38/exec/java-exec/src/test/resources/queries/tpch/18.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/18.sql b/exec/java-exec/src/test/resources/queries/tpch/18.sql
index 6fcddeb..33e4b36 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/18.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/18.sql
@@ -18,7 +18,7 @@ where
       cp.`tpch/lineitem.parquet`
     group by
       l_orderkey having
-        sum(l_quantity) > 313
+        sum(l_quantity) > 300
   )
   and c.c_custkey = o.o_custkey
   and o.o_orderkey = l.l_orderkey


[12/51] [abbrv] git commit: in partition sender, on Iteroutcome.STOP, cleanup without flushing

Posted by ja...@apache.org.
in partition sender, on Iteroutcome.STOP, cleanup without flushing


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

Branch: refs/heads/master
Commit: e28e1ef346b2e29862fb760be973c9145b55b761
Parents: bf3fa66
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 17:47:56 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../impl/partitionsender/PartitionSenderRootExec.java         | 7 ++++++-
 1 file changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e28e1ef3/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index 19fe98b..d0efcb0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -95,7 +95,6 @@ public class PartitionSenderRootExec implements RootExec {
     logger.debug("Partitioner.next(): got next record batch with status {}", out);
     switch(out){
       case NONE:
-      case STOP:
         try {
           // send any pending batches
           for (OutgoingRecordBatch batch : outgoing) {
@@ -109,6 +108,12 @@ public class PartitionSenderRootExec implements RootExec {
         }
         return false;
 
+      case STOP:
+        for (OutgoingRecordBatch batch : outgoing) {
+          batch.clear();
+        }
+        return false;
+
       case OK_NEW_SCHEMA:
         try {
           // send all existing batches


[30/51] [abbrv] git commit: Fix the duplicate field names in join operator. Work in progress for column star.

Posted by ja...@apache.org.
Fix the duplicate field names in join operator.  Work in progress for column star.


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

Branch: refs/heads/master
Commit: d91a01a122fa91bed06cafa4a53aaf99770e8423
Parents: 31d1994
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Wed Apr 2 12:12:13 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../physical/config/SelectionVectorRemover.java |  7 +-
 .../exec/planner/physical/MergeJoinPrel.java    | 73 +++++++++++++++-----
 .../planner/types/RelDataTypeDrillImpl.java     |  3 +-
 .../exec/planner/types/RelDataTypeHolder.java   | 16 +++--
 4 files changed, 76 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d91a01a1/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
index 7361503..5e891ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/config/SelectionVectorRemover.java
@@ -22,6 +22,7 @@ import org.apache.drill.exec.physical.base.AbstractSingle;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.base.PhysicalVisitor;
 import org.apache.drill.exec.physical.base.Size;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
@@ -56,5 +57,9 @@ public class SelectionVectorRemover extends AbstractSingle {
   public Size getSize() {
     return new Size( (long) (child.getSize().getRecordCount()), child.getSize().getRecordSize());
   }
-  
+
+  @Override
+  public SelectionVectorMode getSVMode() {
+    return SelectionVectorMode.NONE;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d91a01a1/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index f3b893c..bfb2192 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -18,15 +18,14 @@
 package org.apache.drill.exec.planner.physical;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.List;
 
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.JoinCondition;
-import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.MergeJoinPOP;
+import org.apache.drill.exec.physical.config.Project;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.planner.common.DrillJoinRelBase;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -59,7 +58,6 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     if (!remaining.isAlwaysTrue()) {
       throw new InvalidRelException("MergeJoinPrel only supports equi-join");
     }
-    //this.joinConditions = joinConditions;
   }
 
   
@@ -74,16 +72,21 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
 
   @Override  
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {    
-    PhysicalOperator leftPop = ((Prel) getLeft()).getPhysicalOperator(creator);
+    final List<String> fields = getRowType().getFieldNames();
+    assert isUnique(fields);
+    final int leftCount = left.getRowType().getFieldCount();
+    final List<String> leftFields = fields.subList(0, leftCount);
+    final List<String> rightFields = fields.subList(leftCount, fields.size());
 
+    PhysicalOperator leftPop = implementInput(creator, 0, left);
+    PhysicalOperator rightPop = implementInput(creator, leftCount, right);
+   
     //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
     if (leftPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
       leftPop = new SelectionVectorRemover(leftPop);
       creator.addPhysicalOperator(leftPop);
     }
 
-    PhysicalOperator rightPop = ((Prel) getRight()).getPhysicalOperator(creator);
-
     //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
     if (rightPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
       rightPop = new SelectionVectorRemover(rightPop);
@@ -91,18 +94,13 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     }
     
     JoinRelType jtype = this.getJoinType();
-    
-    final List<String> fields = getRowType().getFieldNames();
-    assert isUnique(fields);
-    final int leftCount = left.getRowType().getFieldCount();
-    final List<String> leftFields = fields.subList(0, leftCount);
-    final List<String> rightFields = fields.subList(leftCount, fields.size());
-    
+            
     List<JoinCondition> conditions = Lists.newArrayList();
     
     for (Pair<Integer, Integer> pair : Pair.zip(leftKeys, rightKeys)) {
       conditions.add(new JoinCondition("==", new FieldReference(leftFields.get(pair.left)), new FieldReference(rightFields.get(pair.right))));
     }
+    
     MergeJoinPOP mjoin = new MergeJoinPOP(leftPop, rightPop, conditions, jtype);
     creator.addPhysicalOperator(mjoin);
    
@@ -117,7 +115,48 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     return this.rightKeys;
   }
   
-//  public JoinCondition[] getJoinConditions() {
-//    return joinConditions;
-//  }
+  /**
+   * Check to make sure that the fields of the inputs are the same as the output field names.  If not, insert a project renaming them.
+   * @param implementor
+   * @param i
+   * @param offset
+   * @param input
+   * @return
+   */
+  private PhysicalOperator implementInput(PhysicalPlanCreator creator, int offset, RelNode input) throws IOException {
+    final PhysicalOperator inputOp = ((Prel) input).getPhysicalOperator(creator); 
+    assert uniqueFieldNames(input.getRowType());
+    final List<String> fields = getRowType().getFieldNames();
+    final List<String> inputFields = input.getRowType().getFieldNames();
+    final List<String> outputFields = fields.subList(offset, offset + inputFields.size());
+    if (!outputFields.equals(inputFields)) {
+      // Ensure that input field names are the same as output field names.
+      // If there are duplicate field names on left and right, fields will get
+      // lost.
+      return rename(creator, inputOp, inputFields, outputFields);
+    } else {
+      return inputOp;
+    }
+  }
+
+  private PhysicalOperator rename(PhysicalPlanCreator creator, PhysicalOperator inputOp, List<String> inputFields, List<String> outputFields) {
+    List<NamedExpression> exprs = Lists.newArrayList();
+    
+    //Currently, Project only accepts "NONE". For other, requires SelectionVectorRemover
+    if (!inputOp.getSVMode().equals(SelectionVectorMode.NONE)) {
+      inputOp = new SelectionVectorRemover(inputOp);
+      creator.addPhysicalOperator(inputOp);
+    }
+
+    for (Pair<String, String> pair : Pair.zip(inputFields, outputFields)) {
+      exprs.add(new NamedExpression(new FieldReference(pair.left), new FieldReference("output." + pair.right)));
+    }
+    
+    Project proj = new Project(exprs, inputOp);
+    
+    creator.addPhysicalOperator(proj);
+    return proj;
+  }
+
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d91a01a1/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
index 8b031ec..0f3c24f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeDrillImpl.java
@@ -41,6 +41,7 @@ public class RelDataTypeDrillImpl extends RelDataTypeImpl {
     public RelDataTypeDrillImpl(RelDataTypeHolder holder, RelDataTypeFactory typeFactory) {
         this.typeFactory = typeFactory;
         this.holder = holder;
+        this.holder.setRelDataTypeFactory(typeFactory);
         computeDigest();
     }
     
@@ -76,7 +77,7 @@ public class RelDataTypeDrillImpl extends RelDataTypeImpl {
 
     @Override
     protected void generateTypeString(StringBuilder sb, boolean withDetail) {
-       sb.append("DrillRecordRow");
+       sb.append("(DrillRecordRow" + getFieldNames() + ")");
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d91a01a1/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
index 939e9ac..8515b0c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
@@ -31,19 +31,22 @@ public class RelDataTypeHolder {
 
   List<RelDataTypeField> fields = Lists.newArrayList();
 
+  private RelDataTypeFactory typeFactory;
+  
   public List<RelDataTypeField> getFieldList(RelDataTypeFactory typeFactory) {
     
-    addStarIfEmpty();
+    addStarIfEmpty(typeFactory);
     return fields;
   }
 
   public int getFieldCount() {
-    addStarIfEmpty();
+    addStarIfEmpty(this.typeFactory);
     return fields.size();
   }
 
-  private void addStarIfEmpty(){
-    //if (fieldNames.isEmpty()) fieldNames.add("*");
+  private void addStarIfEmpty(RelDataTypeFactory typeFactory){
+//    RelDataTypeField starCol = getField(typeFactory, "*");
+//    if (fields.isEmpty()) fields.add(starCol);
   }
   
   public RelDataTypeField getField(RelDataTypeFactory typeFactory, String fieldName) {
@@ -72,4 +75,9 @@ public class RelDataTypeHolder {
     
     return fieldNames;
   }
+  
+  public void setRelDataTypeFactory(RelDataTypeFactory typeFactory) {
+    this.typeFactory = typeFactory;
+  }
+  
 }


[11/51] [abbrv] git commit: in partition sender, release incoming vectors after partitioning

Posted by ja...@apache.org.
in partition sender, release incoming vectors after partitioning


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

Branch: refs/heads/master
Commit: d37d9241e63a0160cd8d03610326c927b60ea1d0
Parents: e28e1ef
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 19:45:10 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../physical/impl/partitionsender/PartitionSenderRootExec.java    | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/d37d9241/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
index d0efcb0..3e3157b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/PartitionSenderRootExec.java
@@ -133,6 +133,9 @@ public class PartitionSenderRootExec implements RootExec {
         }
       case OK:
         partitioner.partitionBatch(incoming);
+        for (VectorWrapper v : incoming) {
+          v.clear();
+        }
         context.getStats().batchesCompleted.inc(1);
         context.getStats().recordsCompleted.inc(incoming.getRecordCount());
         return true;


[17/51] [abbrv] git commit: serialize "and" function as "booleanAnd" to not confuse parser

Posted by ja...@apache.org.
serialize "and" function as "booleanAnd" to not confuse parser


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

Branch: refs/heads/master
Commit: 8c51248948474c2fc114b23d2f1521e8734b934f
Parents: 18a24da
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 15:57:12 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/common/expression/FunctionCallFactory.java   | 3 ++-
 .../java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java     | 2 +-
 2 files changed, 3 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8c512489/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
index 71fd8bd..d1f449c 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
@@ -47,7 +47,8 @@ public class FunctionCallFactory {
     opToFuncTable.put("^", "xor");
 
     opToFuncTable.put("||", "or");
-    opToFuncTable.put("&&", "and");
+    opToFuncTable.put("and", "booleanAnd");
+    opToFuncTable.put("&&", "booleanAnd");
     opToFuncTable.put(">", "greater_than");
     opToFuncTable.put("<", "less_than");
     opToFuncTable.put("==", "equal");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/8c512489/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
index 9b75574..802595a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
@@ -43,7 +43,7 @@ public class BitFunctions {
     }
   }  
 
-  @FunctionTemplate(names = {"and", "&&"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = {"booleanAnd", "and", "&&"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class BitAnd implements DrillSimpleFunc {
 
     @Param BitHolder left;


[41/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
index 9b2f892..25b62ad 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/TopN/TestSimpleTopN.java
@@ -59,8 +59,8 @@ import static org.junit.Assert.assertTrue;
 public class TestSimpleTopN extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleTopN.class);
   DrillConfig c = DrillConfig.create();
-  
-  
+
+
   @Test
   public void sortOneKeyAscending() throws Throwable{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
@@ -103,14 +103,16 @@ public class TestSimpleTopN extends PopUnitTestBase {
           assertTrue(previousBigInt <= a1.get(i));
           previousBigInt = a1.get(i);
         }
+        loader.clear();
+        b.release();
       }
 
       System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));
 
     }
-    
+
 
   }
-  
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
index 66219e8..938a479 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/agg/TestAgg.java
@@ -52,14 +52,14 @@ import com.codahale.metrics.MetricRegistry;
 public class TestAgg {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAgg.class);
   DrillConfig c = DrillConfig.create();
-  
+
   private SimpleRootExec doTest(final DrillbitContext bitContext, UserClientConnection connection, String file) throws Exception{
     new NonStrictExpectations(){{
       bitContext.getMetrics(); result = new MetricRegistry();
       bitContext.getAllocator(); result = new TopLevelAllocator();
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
-    
+
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile(file), Charsets.UTF_8));
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
@@ -67,44 +67,44 @@ public class TestAgg {
     SimpleRootExec exec = new SimpleRootExec(ImplCreator.getExec(context, (FragmentRoot) plan.getSortedOperators(false).iterator().next()));
     return exec;
   }
-  
+
   @Test
   public void oneKeyAgg(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{
     SimpleRootExec exec = doTest(bitContext, connection, "/agg/test1.json");
-    
+
     while(exec.next()){
-      BigIntVector cnt = exec.getValueVectorById(new SchemaPath("cnt", ExpressionPosition.UNKNOWN), BigIntVector.class);
-      IntVector key = exec.getValueVectorById(new SchemaPath("blue", ExpressionPosition.UNKNOWN), IntVector.class);
+      BigIntVector cnt = exec.getValueVectorById(SchemaPath.getSimplePath("cnt"), BigIntVector.class);
+      IntVector key = exec.getValueVectorById(SchemaPath.getSimplePath("blue"), IntVector.class);
       long[] cntArr = {10001, 9999};
       int[] keyArr = {Integer.MIN_VALUE, Integer.MAX_VALUE};
-      
+
       for(int i =0; i < exec.getRecordCount(); i++){
         assertEquals(cntArr[i], cnt.getAccessor().getObject(i));
         assertEquals(keyArr[i], key.getAccessor().getObject(i));
       }
     }
-    
+
     if(exec.getContext().getFailureCause() != null){
       throw exec.getContext().getFailureCause();
     }
     assertTrue(!exec.getContext().isFailed());
 
   }
-  
+
   @Test
   public void twoKeyAgg(@Injectable final DrillbitContext bitContext, @Injectable UserClientConnection connection) throws Throwable{
     SimpleRootExec exec = doTest(bitContext, connection, "/agg/twokey.json");
-    
+
     while(exec.next()){
-      IntVector key1 = exec.getValueVectorById(new SchemaPath("key1", ExpressionPosition.UNKNOWN), IntVector.class);
-      BigIntVector key2 = exec.getValueVectorById(new SchemaPath("key2", ExpressionPosition.UNKNOWN), BigIntVector.class);
-      BigIntVector cnt = exec.getValueVectorById(new SchemaPath("cnt", ExpressionPosition.UNKNOWN), BigIntVector.class);
-      BigIntVector total = exec.getValueVectorById(new SchemaPath("total", ExpressionPosition.UNKNOWN), BigIntVector.class);
+      IntVector key1 = exec.getValueVectorById(SchemaPath.getSimplePath("key1"), IntVector.class);
+      BigIntVector key2 = exec.getValueVectorById(SchemaPath.getSimplePath("key2"), BigIntVector.class);
+      BigIntVector cnt = exec.getValueVectorById(SchemaPath.getSimplePath("cnt"), BigIntVector.class);
+      BigIntVector total = exec.getValueVectorById(SchemaPath.getSimplePath("total"), BigIntVector.class);
       int[] keyArr1 = {Integer.MIN_VALUE, Integer.MIN_VALUE, Integer.MIN_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE, Integer.MAX_VALUE};
       long[] keyArr2 = {0,1,2,0,1,2};
       long[] cntArr = {34,34,34,34,34,34};
       long[] totalArr = {0,34,68,0,34,68};
-      
+
       for(int i =0; i < exec.getRecordCount(); i++){
 //        System.out.print(key1.getAccessor().getObject(i));
 //        System.out.print("\t");
@@ -120,14 +120,14 @@ public class TestAgg {
         assertEquals(totalArr[i], total.getAccessor().getObject(i));
       }
     }
-    
+
     if(exec.getContext().getFailureCause() != null){
       throw exec.getContext().getFailureCause();
     }
     assertTrue(!exec.getContext().isFailed());
 
   }
-  
+
   @AfterClass
   public static void tearDown() throws Exception{
     // pause to get logger to catch up.

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
index 1623b86..f5c9884 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoin.java
@@ -87,7 +87,7 @@ public class TestMergeJoin extends PopUnitTestBase {
     while (exec.next()) {
       totalRecordCount += exec.getRecordCount();
       for (ValueVector v : exec)
-        System.out.print("[" + v.getField().getName() + "]        ");
+        System.out.print("[" + v.getField().toExpr() + "]        ");
       System.out.println("\n");
       for (int valueIdx = 0; valueIdx < exec.getRecordCount(); valueIdx++) {
         List<Object> row = new ArrayList();
@@ -95,7 +95,7 @@ public class TestMergeJoin extends PopUnitTestBase {
            row.add(v.getAccessor().getObject(valueIdx));
         }
         for (Object cell : row) {
-          if (cell == null) { 
+          if (cell == null) {
             System.out.print("<null>          ");
             continue;
           }
@@ -142,11 +142,11 @@ public class TestMergeJoin extends PopUnitTestBase {
       totalRecordCount += exec.getRecordCount();
       System.out.println("got next with record count: " + exec.getRecordCount() + " (total: " + totalRecordCount + "):");
       System.out.println("       t1                 t2");
-                          
+
       for (int valueIdx = 0; valueIdx < exec.getRecordCount(); valueIdx++) {
         List<Object> row = Lists.newArrayList();
         for (ValueVector v : exec)
-          row.add(v.getField().getName() + ":" + v.getAccessor().getObject(valueIdx));
+          row.add(v.getField().toExpr() + ":" + v.getAccessor().getObject(valueIdx));
         for (Object cell : row) {
           if (cell == null) {
             System.out.print("<null>    ");
@@ -199,7 +199,7 @@ public class TestMergeJoin extends PopUnitTestBase {
       for (int valueIdx = 0; valueIdx < exec.getRecordCount(); valueIdx++) {
         List<Object> row = Lists.newArrayList();
         for (ValueVector v : exec)
-          row.add(v.getField().getName() + ":" + v.getAccessor().getObject(valueIdx));
+          row.add(v.getField().toExpr() + ":" + v.getAccessor().getObject(valueIdx));
         for (Object cell : row) {
           if (cell == null) {
             System.out.print("<null>    ");
@@ -251,7 +251,7 @@ public class TestMergeJoin extends PopUnitTestBase {
       for (int valueIdx = 0; valueIdx < exec.getRecordCount(); valueIdx++) {
         List<Object> row = Lists.newArrayList();
         for (ValueVector v : exec)
-          row.add(v.getField().getName() + ":" + v.getAccessor().getObject(valueIdx));
+          row.add(v.getField().toExpr() + ":" + v.getAccessor().getObject(valueIdx));
         for (Object cell : row) {
           if (cell == null) {
             System.out.print("<null>    ");
@@ -282,7 +282,7 @@ public class TestMergeJoin extends PopUnitTestBase {
       bitContext.getConfig(); result = c;
       bitContext.getOperatorCreatorRegistry(); result = new OperatorCreatorRegistry(c);
     }};
-    
+
     PhysicalPlanReader reader = new PhysicalPlanReader(c, c.getMapper(), CoordinationProtos.DrillbitEndpoint.getDefaultInstance());
     PhysicalPlan plan = reader.readPhysicalPlan(Files.toString(FileUtils.getResourceAsFile("/join/join_batchsize.json"), Charsets.UTF_8));
     FunctionImplementationRegistry registry = new FunctionImplementationRegistry(c);
@@ -291,7 +291,7 @@ public class TestMergeJoin extends PopUnitTestBase {
     while(exec.next()){
       assertEquals(100, exec.getRecordCount());
     }
-    
+
     if(context.getFailureCause() != null){
       throw context.getFailureCause();
     }
@@ -325,7 +325,7 @@ public class TestMergeJoin extends PopUnitTestBase {
   public void testMergeJoinLeftEmptyBatch() throws Exception {
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
-    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);      
+    try(Drillbit bit1 = new Drillbit(CONFIG, serviceSet);
         DrillClient client = new DrillClient(CONFIG, serviceSet.getCoordinator());) {
 
       bit1.run();
@@ -371,7 +371,7 @@ public class TestMergeJoin extends PopUnitTestBase {
     Thread.sleep(1000);
   }
 
-  
+
 }
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
index 24dbe71..ebab889 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/join/TestMergeJoinMulCondition.java
@@ -38,7 +38,7 @@ import com.google.common.io.Files;
 
 public class TestMergeJoinMulCondition extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestMergeJoinMulCondition.class);
-  
+
   @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(200000);
 
   @Test
@@ -47,7 +47,7 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
   //      + " from cp.`tpch/lineitem.parquet` l join "
   //      + "      cp.`tpch/partsupp.parquet` ps"
   //      + " on l.l_partkey = ps.ps_partkey and "
-  //      + "    l.l_suppkey = ps.ps_suppkey";    
+  //      + "    l.l_suppkey = ps.ps_suppkey";
   public void testMergeJoinMultiKeys() throws Exception {
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
 
@@ -61,11 +61,13 @@ public class TestMergeJoinMulCondition extends PopUnitTestBase {
               Charsets.UTF_8));
       int count = 0;
       for(QueryResultBatch b : results) {
-        if (b.getHeader().getRowCount() != 0)
+        if (b.getHeader().getRowCount() != 0){
           count += b.getHeader().getRowCount();
+        }
+        b.release();
       }
       assertEquals(60175, count);
     }
   }
-    
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
index d1061de..b05c778 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/mergereceiver/TestMergingReceiver.java
@@ -66,7 +66,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
           List<Object> row = Lists.newArrayList();
           batchLoader.load(b.getHeader().getDef(), b.getData());
           for (VectorWrapper<?> vw : batchLoader)
-            row.add(vw.getValueVector().getField().getName() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
+            row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
           for (Object cell : row) {
             if (cell == null) {
               System.out.print("<null>    ");
@@ -79,6 +79,8 @@ public class TestMergingReceiver extends PopUnitTestBase {
           }
           System.out.println();
         }
+        b.release();
+        batchLoader.clear();
       }
       assertEquals(200, count);
     }
@@ -108,8 +110,8 @@ public class TestMergingReceiver extends PopUnitTestBase {
           List<Object> row = Lists.newArrayList();
           batchLoader.load(b.getHeader().getDef(), b.getData());
           for (VectorWrapper vw : batchLoader) {
-            row.add(vw.getValueVector().getField().getName() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
-            if (vw.getValueVector().getField().getName().equals("blue")) {
+            row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
+            if (vw.getValueVector().getField().getAsSchemaPath().getRootSegment().getPath().equals("blue")) {
               // assert order is ascending
               if (((Long)vw.getValueVector().getAccessor().getObject(valueIdx)).longValue() == 0) continue; // ignore initial 0's from sort
               if (lastBlueValue != null)
@@ -125,6 +127,8 @@ public class TestMergingReceiver extends PopUnitTestBase {
           }
           System.out.println();
         }
+        b.release();
+        batchLoader.clear();
       }
       assertEquals(400, count);
     }
@@ -153,7 +157,7 @@ public class TestMergingReceiver extends PopUnitTestBase {
           List<Object> row = Lists.newArrayList();
           batchLoader.load(b.getHeader().getDef(), b.getData());
           for (VectorWrapper vw : batchLoader)
-            row.add(vw.getValueVector().getField().getName() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
+            row.add(vw.getValueVector().getField().toExpr() + ":" + vw.getValueVector().getAccessor().getObject(valueIdx));
           for (Object cell : row) {
             if (cell == null) {
               System.out.print("<null>    ");
@@ -166,6 +170,8 @@ public class TestMergingReceiver extends PopUnitTestBase {
           }
           System.out.println();
         }
+        b.release();
+        batchLoader.clear();
       }
       assertEquals(100, count);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
index 1746d56..699f075 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/orderedpartitioner/TestOrderedPartitionExchange.java
@@ -113,7 +113,10 @@ public class TestOrderedPartitionExchange extends PopUnitTestBase {
             }
           }
           partitionRecordCounts.add(partitionRecordCount);
+          loader.clear();
         }
+
+        b.release();
       }
       double[] values = new double[partitionRecordCounts.size()];
       int i = 0;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
index 8b3d36e..b17f7e7 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/physical/impl/xsort/TestSimpleExternalSort.java
@@ -41,8 +41,8 @@ import static org.junit.Assert.assertTrue;
 public class TestSimpleExternalSort extends PopUnitTestBase {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSimpleExternalSort.class);
   DrillConfig c = DrillConfig.create();
-  
-  
+
+
   @Test
   public void sortOneKeyDescendingMergeSort() throws Throwable{
     RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
@@ -85,6 +85,8 @@ public class TestSimpleExternalSort extends PopUnitTestBase {
           assertTrue(String.format("%d > %d", previousBigInt, a1.get(i)), previousBigInt >= a1.get(i));
           previousBigInt = a1.get(i);
         }
+        loader.clear();
+        b.release();
       }
 
       System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));
@@ -136,6 +138,8 @@ public class TestSimpleExternalSort extends PopUnitTestBase {
           assertTrue(String.format("%d < %d", previousBigInt, a1.get(i)), previousBigInt >= a1.get(i));
           previousBigInt = a1.get(i);
         }
+        loader.clear();
+        b.release();
       }
 
       System.out.println(String.format("Sorted %,d records in %d batches.", recordCount, batchCount));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
index 5595b72..544a4d2 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestDateTypes.java
@@ -99,6 +99,11 @@ public class TestDateTypes extends PopUnitTestBase {
                 assertTrue((accessor.getObject(1)).equals("2008-12-28"));
                 assertTrue((accessor.getObject(2)).equals("2000-02-27"));
             }
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 
@@ -130,6 +135,11 @@ public class TestDateTypes extends PopUnitTestBase {
                 assertEquals((accessor.getObject(1)), new String("2000-02-27"));
                 assertEquals((accessor.getObject(2)), new String("2008-12-28"));
             }
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 
@@ -161,6 +171,11 @@ public class TestDateTypes extends PopUnitTestBase {
                 assertTrue((accessor.getObject(1)).equals("2008-12-28 11:34:00.129"));
                 assertTrue((accessor.getObject(2)).equals("2000-02-27 14:24:00.000"));
             }
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 
@@ -213,6 +228,11 @@ public class TestDateTypes extends PopUnitTestBase {
             assertTrue((accessor.getObject(3)).equals("1 day 1:20:35.897"));
             assertTrue((accessor.getObject(4)).equals("0 days 0:0:35.4"));
             assertTrue((accessor.getObject(5)).equals("1 day 0:-39:-25.0"));
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 
@@ -250,6 +270,11 @@ public class TestDateTypes extends PopUnitTestBase {
 
                 assertTrue((accessor.getObject(0)).equals(result[idx++]));
             }
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 
@@ -280,6 +305,11 @@ public class TestDateTypes extends PopUnitTestBase {
                 assertTrue((accessor.getObject(0)).equals("2008-03-27"));
 
             }
+
+            batchLoader.clear();
+            for(QueryResultBatch b : results){
+              b.release();
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java
index bc7fa90..6251c7f 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/record/vector/TestLoad.java
@@ -73,7 +73,7 @@ public class TestLoad {
       index += byteBufs[i].writerIndex();
     }
     byteBuf.writerIndex(bytes);
-    
+
     batchLoader.load(writableBatch.getDef(), byteBuf);
     boolean firstColumn = true;
     int recordCount = 0;
@@ -83,7 +83,7 @@ public class TestLoad {
       } else {
         System.out.print("\t");
       }
-      System.out.print(v.getField().getName());
+      System.out.print(v.getField().toExpr());
       System.out.print("[");
       System.out.print(v.getField().getType().getMinorType());
       System.out.print("]");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
index 429d5fe..ffe05a4 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOrphanSchema.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.store;
 
 import mockit.NonStrictExpectations;
 import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.memory.TopLevelAllocator;
@@ -29,12 +30,12 @@ import com.codahale.metrics.MetricRegistry;
 
 public class TestOrphanSchema {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestOrphanSchema.class);
-  
-  
+
+
   @Test
   public void test(final DrillbitContext bitContext){
     final DrillConfig c = DrillConfig.create();
-    
+
     new NonStrictExpectations() {
       {
         bitContext.getMetrics();
@@ -45,14 +46,15 @@ public class TestOrphanSchema {
         result = c;
       }
     };
-    
+
     StoragePluginRegistry r = new StoragePluginRegistry(bitContext);
-    SchemaPlus plus = r.getSchemaFactory().getOrphanedRootSchema();
-    
+    SchemaPlus plus = Frameworks.createRootSchema();
+    r.getSchemaFactory().registerSchemas(null, plus);
+
     printSchema(plus, 0);
-        
+
   }
-  
+
   private static void t(final int t){
     for(int i =0; i < t; i++) System.out.print('\t');
   }
@@ -65,11 +67,11 @@ public class TestOrphanSchema {
       System.out.print("Table: ");
       System.out.println(table);
     }
-    
+
     for(String schema : s.getSubSchemaNames()){
       SchemaPlus p = s.getSubSchema(schema);
       printSchema(p, indent + 1);
     }
-    
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOutputMutator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOutputMutator.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOutputMutator.java
index 51a0b0b..37369da 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOutputMutator.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/TestOutputMutator.java
@@ -21,6 +21,8 @@ import java.util.Iterator;
 import java.util.Map;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.expr.TypeHelper;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.apache.drill.exec.record.MaterializedField;
@@ -35,6 +37,11 @@ public class TestOutputMutator implements OutputMutator, Iterable<VectorWrapper<
 
   private final VectorContainer container = new VectorContainer();
   private final Map<MaterializedField, ValueVector> fieldVectorMap = Maps.newHashMap();
+  private final BufferAllocator allocator;
+
+  public TestOutputMutator(BufferAllocator allocator){
+    this.allocator = allocator;
+  }
 
   public void removeField(MaterializedField field) throws SchemaChangeException {
     ValueVector vector = fieldVectorMap.remove(field);
@@ -71,4 +78,12 @@ public class TestOutputMutator implements OutputMutator, Iterable<VectorWrapper<
     removeAllFields();
   }
 
+  @Override
+  public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+    ValueVector v = TypeHelper.getNewVector(field, allocator);
+    if(!clazz.isAssignableFrom(v.getClass())) throw new SchemaChangeException(String.format("The class that was provided %s does not correspond to the expected vector type of %s.", clazz.getSimpleName(), v.getClass().getSimpleName()));
+    addField(v);
+    return (T) v;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
index ee36493..22e4a14 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/OrphanSchema.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.store.ischema;
 
 import static org.mockito.Mockito.*;
 import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.memory.TopLevelAllocator;
@@ -27,49 +28,43 @@ import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.junit.Test;
 
-
 import com.codahale.metrics.MetricRegistry;
-/**
- * OrphanSchema is a stand alone schema tree which is not connected to Optiq.
- * This class is a refactoring of exec.store.TestOrphanSchema.java. The primary
- * change is to package a "create()" method for providing a test schema. 
- * For convenient testing, it mocks up the Drillbit context.
- */
+
 public class OrphanSchema {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrphanSchema.class);
-  
+
   /**
    * Create an orphan schema to be used for testing.
    * @return root node of the created schema.
    */
   public static SchemaPlus create(){
-    
+
     final DrillConfig c = DrillConfig.create();
-    
+
     // Mock up a context which will allow us to create a schema.
     final DrillbitContext bitContext = mock(DrillbitContext.class);
     when(bitContext.getMetrics()).thenReturn(new MetricRegistry());
     when(bitContext.getAllocator()).thenReturn(new TopLevelAllocator());
     when(bitContext.getConfig()).thenReturn(c);
-    
+
     // Using the mock context, get the orphan schema.
     StoragePluginRegistry r = new StoragePluginRegistry(bitContext);
-    SchemaPlus plus = r.getSchemaFactory().getOrphanedRootSchema();
-
+    SchemaPlus plus = Frameworks.createRootSchema();
+    r.getSchemaFactory().registerSchemas(null, plus);
     return plus;
   }
-  
-  
+
+
   /**
    * This test replicates the one in org.apache.drill.exec.server,
    * but it is refactored to provide a standalone "create()" method.
    */
-  
+
   @Test
   public void test() {
     printSchema(create(), 0);
   }
-  
+
   private static void t(final int t){
     for(int i =0; i < t; i++) System.out.print('\t');
   }
@@ -82,11 +77,11 @@ public class OrphanSchema {
       System.out.print("Table: ");
       System.out.println(table);
     }
-    
+
     for(String schema : s.getSubSchemaNames()){
       SchemaPlus p = s.getSubSchema(schema);
       printSchema(p, indent + 1);
     }
-    
+
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestOrphanSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestOrphanSchema.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestOrphanSchema.java
index c29c05f..e76c609 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestOrphanSchema.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestOrphanSchema.java
@@ -52,48 +52,48 @@ public class TestOrphanSchema {
   public void testTables() {
     displayTable(new InfoSchemaTable.Tables(), new OptiqProvider.Tables(root));
   }
-  
+
   @Test
   public void testSchemata() {
     displayTable(new InfoSchemaTable.Schemata(), new OptiqProvider.Schemata(root));
   }
-  
-  
+
+
   @Test
   public void testViews() {
     displayTable(new InfoSchemaTable.Views(), new OptiqProvider.Views(root));
   }
-  
+
   @Test
   public void testCatalogs() {
     displayTable(new InfoSchemaTable.Catalogs(), new OptiqProvider.Catalogs(root));
   }
-  
+
   @Test
   public void testColumns() {
     displayTable(new InfoSchemaTable.Columns(), new OptiqProvider.Columns(root));
   }
-  
-  
+
+
   private void displayTable(FixedTable table, RowProvider provider) {
 
     // Set up a mock context
     FragmentContext context = mock(FragmentContext.class);
     when(context.getAllocator()).thenReturn(new TopLevelAllocator());
-    
+
     // Create a RecordReader which reads from the test table.
     RecordReader reader = new RowRecordReader(context, table, provider);
-    
+
     // Create an dummy output mutator for the RecordReader.
     TestOutput output = new TestOutput();
     try {reader.setup(output);}
     catch (ExecutionSetupException e) {Assert.fail("reader threw an exception");}
-    
+
     // print out headers
     System.out.printf("\n%20s\n", table.getName());
     System.out.printf("%10s", "RowNumber");
     for (ValueVector v: table.getValueVectors()) {
-      System.out.printf(" | %16s", v.getField().getName());
+      System.out.printf(" | %16s", v.getField().toExpr());
     }
     System.out.println();
 
@@ -102,40 +102,45 @@ public class TestOrphanSchema {
     for (;;) {
       int count = reader.next();
       if (count == 0) break;
-      
+
       // Do for each row in the batch
       for (int row=0; row<count; row++, rowNumber++) {
-       
+
         // Display the row
         System.out.printf("%10d", rowNumber);
         for (ValueVector v: table.getValueVectors()) {
           System.out.printf(" | %16s", v.getAccessor().getObject(row));
         }
         System.out.println();
-        
+
       }
     }
   }
 
-  
-  /** 
-   * A dummy OutputMutator so we can examine the contents of the current batch 
+
+  /**
+   * A dummy OutputMutator so we can examine the contents of the current batch
    */
   static class TestOutput implements OutputMutator {
     List<ValueVector> vectors = new ArrayList<ValueVector>();
 
     public void addField(ValueVector vector) throws SchemaChangeException {
-      vectors.add(vector); 
+      vectors.add(vector);
     }
-    
+
     public Object get(int column, int row) {
       return vectors.get(column).getAccessor().getObject(row);
     }
-     
+
     public void removeField(MaterializedField field) {}
     public void removeAllFields() {}
     public void setNewSchema() {}
+
+    @Override
+    public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+      return null;
+    }
   }
-  
- 
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestTableProvider.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestTableProvider.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestTableProvider.java
index 475d2ac..0cc0b71 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestTableProvider.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/ischema/TestTableProvider.java
@@ -40,42 +40,42 @@ import org.junit.Test;
  * Using a test table with two columns, create data and verify the values are in the record batch.
  */
 public class TestTableProvider {
-  
+
   @Test
   public void zeroRead() {
     readTestTable(0);
   }
-  
+
   @Test
   public void oneRead() {
     readTestTable(1);
   }
-  
+
   @Test
   public void smallRead() {
     readTestTable(10);
   }
-  
+
   @Test
   @Ignore // due to out of heap space
   public void largeRead() {
     readTestTable(1024*1024);
   }
-  
-  
+
+
   /**
    * Read record batches from the test table and verify the contents.
    * @param nrRows - the total number of rows expected.
    */
   private void readTestTable(int nrRows) {
-    
+
     // Mock up a context with a BufferAllocator
     FragmentContext context = mock(FragmentContext.class);
     when(context.getAllocator()).thenReturn(new TopLevelAllocator());
-    
+
     // Create a RecordReader which reads from the test table.
     RecordReader reader = new RowRecordReader(context, new TestTable(), new TestProvider(nrRows));
-    
+
     // Create an dummy output mutator for the RecordReader.
     TestOutput output = new TestOutput();
     try {reader.setup(output);}
@@ -86,10 +86,10 @@ public class TestTableProvider {
     for (;;) {
       int count = reader.next();
       if (count == 0) break;
-      
+
       // Do for each row in the batch
       for (int row=0; row<count; row++, rowNumber++) {
-        
+
         // Verify the row has an integer and string containing the row number
         int intValue = (int)output.get(1, row);
         String strValue = (String)output.get(0, row);
@@ -97,12 +97,12 @@ public class TestTableProvider {
         Assert.assertEquals(rowNumber, Integer.parseInt(strValue));
       }
     }
-  
+
   // Verify we read the correct number of rows.
   Assert.assertEquals(nrRows, rowNumber);
   }
 
-  
+
   /**
    * Class to define the table we want to create. Two columns - string, integer
    */
@@ -114,8 +114,8 @@ public class TestTableProvider {
       super(tableName, fieldNames, fieldTypes);
     }
   }
-  
-  
+
+
   /**
    * Class to generate data for the table
    */
@@ -130,28 +130,33 @@ public class TestTableProvider {
       }
     }
   }
-  
-  
-  
-  
-  /** 
-   * A dummy OutputMutator so we can examine the contents of the current batch 
+
+
+
+
+  /**
+   * A dummy OutputMutator so we can examine the contents of the current batch
    */
   static class TestOutput implements OutputMutator {
     List<ValueVector> vectors = new ArrayList<ValueVector>();
 
     public void addField(ValueVector vector) throws SchemaChangeException {
-      vectors.add(vector); 
+      vectors.add(vector);
     }
-    
+
     public Object get(int column, int row) {
       return vectors.get(column).getAccessor().getObject(row);
     }
-     
+
     public void removeField(MaterializedField field) {}
     public void removeAllFields() {}
     public void setNewSchema() {}
+
+    @Override
+    public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+      return null;
+    }
   }
-  
- 
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
index 9020c1a..9f54349 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/json/JSONRecordReaderTest.java
@@ -92,6 +92,11 @@ public class JSONRecordReaderTest {
     List<ValueVector> getAddFields() {
       return addFields;
     }
+
+    @Override
+    public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+      return null;
+    }
   }
 
   private <T> void assertField(ValueVector valueVector, int index, MinorType expectedMinorType, T value, String name) {
@@ -268,7 +273,7 @@ public class JSONRecordReaderTest {
     assertField(addFields.get(4), 0, MinorType.VARCHAR, "test2", "str1");
     assertField(addFields.get(5), 0, MinorType.BIGINT, 4L, "d");
     assertEquals(1, removedFields.size());
-    assertEquals("c", removedFields.get(0).getName());
+    assertEquals("c", removedFields.get(0).getAsSchemaPath().getRootSegment().getPath());
     removedFields.clear();
     assertEquals(1, jr.next());
     assertEquals(7, addFields.size()); // The reappearing of field 'c' is also included
@@ -281,19 +286,20 @@ public class JSONRecordReaderTest {
     Iterables.find(removedFields, new Predicate<MaterializedField>() {
       @Override
       public boolean apply(MaterializedField materializedField) {
-        return materializedField.getName().equals("str1");
+        return materializedField.getAsSchemaPath().getRootSegment().getPath().equals("str1");
       }
     });
     Iterables.find(removedFields, new Predicate<MaterializedField>() {
       @Override
       public boolean apply(MaterializedField materializedField) {
-        return materializedField.getName().equals("b");
+        return materializedField.getAsSchemaPath().getRootSegment().getPath().equals("b");
       }
     });
     assertEquals(0, jr.next());
   }
 
   @Test
+  @Ignore // until repeated map
   public void testNestedFieldInSameBatch(@Injectable final FragmentContext context) throws ExecutionSetupException, IOException {
     new Expectations() {
       {
@@ -323,6 +329,7 @@ public class JSONRecordReaderTest {
   }
 
   @Test
+  @Ignore // until repeated map is added.
   public void testRepeatedFields(@Injectable final FragmentContext context) throws ExecutionSetupException, IOException {
     new Expectations() {
       {
@@ -387,7 +394,7 @@ public class JSONRecordReaderTest {
     assertEquals(0, jr.next());
     assertTrue(mutator.getRemovedFields().isEmpty());
   }
-  
+
   @Test
   public void testJsonArrayandNormalFields(@Injectable final FragmentContext context) throws ExecutionSetupException, IOException {
     new Expectations() {
@@ -406,7 +413,7 @@ public class JSONRecordReaderTest {
     jr.setup(mutator);
     assertEquals(2, jr.next());
     assertEquals(3, addFields.size());
-    
+
     assertField(addFields.get(0), 0, MinorType.VARCHAR, "ABC", "test");
     assertField(addFields.get(2), 0, MinorType.VARCHAR, "drill", "a");
     assertField(addFields.get(0), 1, MinorType.VARCHAR, "abc", "test");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
index f6a7d97..9ba94fa 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetRecordReaderTest.java
@@ -30,19 +30,23 @@ import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
 import mockit.Injectable;
+
+import org.apache.drill.BaseTestQuery;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.expression.ExpressionPosition;
-import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.util.FileUtils;
 import org.apache.drill.exec.client.DrillClient;
 import org.apache.drill.exec.exception.SchemaChangeException;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
 import org.apache.drill.exec.ops.FragmentContext;
 import org.apache.drill.exec.physical.impl.OutputMutator;
 import org.apache.drill.exec.proto.BitControl;
 import org.apache.drill.exec.proto.UserProtos;
+import org.apache.drill.exec.proto.UserProtos.QueryType;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.rpc.user.UserServer;
@@ -74,7 +78,7 @@ import com.google.common.base.Stopwatch;
 import com.google.common.collect.Lists;
 import com.google.common.io.Files;
 
-public class ParquetRecordReaderTest {
+public class ParquetRecordReaderTest extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ParquetRecordReaderTest.class);
 
   static boolean VERBOSE_DEBUG = false;
@@ -127,58 +131,36 @@ public class ParquetRecordReaderTest {
     testParquetFullEngineRemote(planName, fileName, 1, numberRowGroups, recordsPerRowGroup);
   }
 
-  
+
   public void testParquetFullEngineLocalPath(String planFileName, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
     testParquetFullEngineLocalText(Files.toString(FileUtils.getResourceAsFile(planFileName), Charsets.UTF_8), filename, numberOfTimesRead, numberOfRowGroups, recordsPerRowGroup);
   }
-  
+
   //specific tests should call this method, but it is not marked as a test itself intentionally
   public void testParquetFullEngineLocalText(String planText, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
-    
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    DrillConfig config = DrillConfig.create();
-
-    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
-      bit1.run();
-      client.connect();
-      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
-      HashMap<String, FieldInfo> fields = new HashMap<>();
-      ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
-      TestFileGenerator.populateFieldInfoMap(props);
-      ParquetResultListener resultListener = new ParquetResultListener(batchLoader, props, numberOfTimesRead, true);
-      Stopwatch watch = new Stopwatch().start();
-      client.runQuery(UserProtos.QueryType.LOGICAL, planText, resultListener);
-      resultListener.getResults();
-      System.out.println(String.format("Took %d ms to run query", watch.elapsed(TimeUnit.MILLISECONDS)));
+    testFull(QueryType.LOGICAL, planText, filename, numberOfTimesRead, numberOfRowGroups, recordsPerRowGroup);
+  }
 
-    }
+  private void testFull(QueryType type, String planText, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
+
+//    RecordBatchLoader batchLoader = new RecordBatchLoader(getAllocator());
+    HashMap<String, FieldInfo> fields = new HashMap<>();
+    ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
+    TestFileGenerator.populateFieldInfoMap(props);
+    ParquetResultListener resultListener = new ParquetResultListener(getAllocator(), props, numberOfTimesRead, true);
+    Stopwatch watch = new Stopwatch().start();
+    testWithListener(type, planText, resultListener);
+    resultListener.getResults();
+//    batchLoader.clear();
+    System.out.println(String.format("Took %d ms to run query", watch.elapsed(TimeUnit.MILLISECONDS)));
 
   }
 
 
   //use this method to submit physical plan
   public void testParquetFullEngineLocalTextDistributed(String planName, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
-
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-
-    DrillConfig config = DrillConfig.create();
-
-    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
-      bit1.run();
-      client.connect();
-      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
-      HashMap<String, FieldInfo> fields = new HashMap<>();
-      ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
-      TestFileGenerator.populateFieldInfoMap(props);
-      ParquetResultListener resultListener = new ParquetResultListener(batchLoader, props, numberOfTimesRead, true);
-      Stopwatch watch = new Stopwatch().start();
-      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(planName), Charsets.UTF_8), resultListener);
-      resultListener.getResults();
-      System.out.println(String.format("Took %d ms to run query", watch.elapsed(TimeUnit.MILLISECONDS)));
-
-    }
-
+    String planText = Files.toString(FileUtils.getResourceAsFile(planName), Charsets.UTF_8);
+    testFull(QueryType.PHYSICAL, planText, filename, numberOfTimesRead, numberOfRowGroups, recordsPerRowGroup);
   }
 
   public String pad(String value, int length) {
@@ -197,20 +179,12 @@ public class ParquetRecordReaderTest {
   }
 
   public void testParquetFullEngineRemote(String plan, String filename, int numberOfTimesRead /* specified in json plan */, int numberOfRowGroups, int recordsPerRowGroup) throws Exception{
-
-    DrillConfig config = DrillConfig.create();
-
-    try(DrillClient client = new DrillClient(config);){
-      client.connect();
-      RecordBatchLoader batchLoader = new RecordBatchLoader(client.getAllocator());
-      HashMap<String, FieldInfo> fields = new HashMap<>();
-      ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
-      TestFileGenerator.populateFieldInfoMap(props);
-      ParquetResultListener resultListener = new ParquetResultListener(batchLoader, props, numberOfTimesRead, true);
-      client.runQuery(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
-      resultListener.getResults();
-    }
-
+    HashMap<String, FieldInfo> fields = new HashMap<>();
+    ParquetTestProperties props = new ParquetTestProperties(numberRowGroups, recordsPerRowGroup, DEFAULT_BYTES_PER_PAGE, fields);
+    TestFileGenerator.populateFieldInfoMap(props);
+    ParquetResultListener resultListener = new ParquetResultListener(getAllocator(), props, numberOfTimesRead, true);
+    testWithListener(UserProtos.QueryType.PHYSICAL, Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8), resultListener);
+    resultListener.getResults();
   }
 
   class MockOutputMutator implements OutputMutator {
@@ -243,6 +217,11 @@ public class ParquetRecordReaderTest {
     List<ValueVector> getAddFields() {
       return addFields;
     }
+
+    @Override
+    public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+      return null;
+    }
   }
 
 
@@ -427,10 +406,11 @@ public class ParquetRecordReaderTest {
     int totalRowCount = 0;
 
     FileSystem fs = new CachedSingleFileSystem(fileName);
+    BufferAllocator allocator = new TopLevelAllocator();
     for(int i = 0; i < 25; i++){
       ParquetRecordReader rr = new ParquetRecordReader(context, 256000, fileName, 0, fs,
           new CodecFactoryExposer(dfsConfig), f.getParquetMetadata(), columns);
-      TestOutputMutator mutator = new TestOutputMutator();
+      TestOutputMutator mutator = new TestOutputMutator(allocator);
       rr.setup(mutator);
       Stopwatch watch = new Stopwatch();
       watch.start();
@@ -442,6 +422,8 @@ public class ParquetRecordReaderTest {
       System.out.println(String.format("Time completed: %s. ", watch.elapsed(TimeUnit.MILLISECONDS)));
       rr.cleanup();
     }
+
+    allocator.close();
     System.out.println(String.format("Total row count %s", totalRowCount));
   }
 
@@ -462,28 +444,23 @@ public class ParquetRecordReaderTest {
   public void testParquetFullEngineEventBased(boolean testValues, boolean generateNew, String plan, String readEntries, String filename,
                                               int numberOfTimesRead /* specified in json plan */, ParquetTestProperties props,
                                               boolean runAsLogicalPlan) throws Exception{
-    RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
     if (generateNew) TestFileGenerator.generateParquetFile(filename, props);
-    DrillConfig config = DrillConfig.create();
-    try(Drillbit bit1 = new Drillbit(config, serviceSet); DrillClient client = new DrillClient(config, serviceSet.getCoordinator());){
-      bit1.run();
-      client.connect();
-      RecordBatchLoader batchLoader = new RecordBatchLoader(bit1.getContext().getAllocator());
-      ParquetResultListener resultListener = new ParquetResultListener(batchLoader, props, numberOfTimesRead, testValues);
-      long C = System.nanoTime();
-      String planText = Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8);
-      // substitute in the string for the read entries, allows reuse of the plan file for several tests
-      if (readEntries != null) {
-        planText = planText.replaceFirst( "&REPLACED_IN_PARQUET_TEST&", readEntries);
-      }
-      if (runAsLogicalPlan)
-        client.runQuery(UserProtos.QueryType.LOGICAL, planText, resultListener);
-      else
-        client.runQuery(UserProtos.QueryType.PHYSICAL, planText, resultListener);
-      resultListener.getResults();
-      long D = System.nanoTime();
-      System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
+
+    ParquetResultListener resultListener = new ParquetResultListener(getAllocator(), props, numberOfTimesRead, testValues);
+    long C = System.nanoTime();
+    String planText = Files.toString(FileUtils.getResourceAsFile(plan), Charsets.UTF_8);
+    // substitute in the string for the read entries, allows reuse of the plan file for several tests
+    if (readEntries != null) {
+      planText = planText.replaceFirst( "&REPLACED_IN_PARQUET_TEST&", readEntries);
+    }
+    if (runAsLogicalPlan){
+      this.testWithListener(QueryType.LOGICAL, planText, resultListener);
+    }else{
+      this.testWithListener(QueryType.PHYSICAL, planText, resultListener);
     }
+    resultListener.getResults();
+    long D = System.nanoTime();
+    System.out.println(String.format("Took %f s to run query", (float)(D-C) / 1E9));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
index f99721a..73af98c 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/ParquetResultListener.java
@@ -24,6 +24,7 @@ import java.util.HashMap;
 
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.proto.UserBitShared;
 import org.apache.drill.exec.record.RecordBatchLoader;
 import org.apache.drill.exec.record.VectorWrapper;
@@ -42,15 +43,16 @@ public class ParquetResultListener implements UserResultsListener {
   private SettableFuture<Void> future = SettableFuture.create();
   int count = 0;
   int totalRecords;
-  RecordBatchLoader batchLoader;
+
   boolean testValues;
+  BufferAllocator allocator;
 
   int batchCounter = 1;
-  HashMap<String, Integer> valuesChecked = new HashMap();
+  HashMap<String, Integer> valuesChecked = new HashMap<>();
   ParquetTestProperties props;
 
-  ParquetResultListener(RecordBatchLoader batchLoader, ParquetTestProperties props, int numberOfTimesRead, boolean testValues){
-    this.batchLoader = batchLoader;
+  ParquetResultListener(BufferAllocator allocator, ParquetTestProperties props, int numberOfTimesRead, boolean testValues){
+    this.allocator = allocator;
     this.props = props;
     this.totalRecords = props.recordsPerRowGroup * props.numberRowGroups * numberOfTimesRead;
     this.testValues = testValues;
@@ -83,7 +85,7 @@ public class ParquetResultListener implements UserResultsListener {
   }
 
   @Override
-  public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+  synchronized public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
     logger.debug("result arrived in test batch listener.");
     if(result.getHeader().getIsLastChunk()){
       future.set(null);
@@ -92,6 +94,7 @@ public class ParquetResultListener implements UserResultsListener {
     FieldInfo currentField;
     count += result.getHeader().getRowCount();
     boolean schemaChanged = false;
+    RecordBatchLoader batchLoader = new RecordBatchLoader(allocator);
     try {
       schemaChanged = batchLoader.load(result.getHeader().getDef(), result.getData());
     } catch (SchemaChangeException e) {
@@ -105,15 +108,15 @@ public class ParquetResultListener implements UserResultsListener {
 
     for (VectorWrapper vw : batchLoader) {
       ValueVector vv = vw.getValueVector();
-      currentField = props.fields.get(vv.getField().getName());
+      currentField = props.fields.get(vv.getField().getAsSchemaPath().getRootSegment().getPath());
       if (ParquetRecordReaderTest.VERBOSE_DEBUG){
         System.out.println("\n" + (String) currentField.name);
       }
-      if ( ! valuesChecked.containsKey(vv.getField().getName())){
-        valuesChecked.put(vv.getField().getName(), 0);
+      if ( ! valuesChecked.containsKey(vv.getField().getAsSchemaPath().getRootSegment().getPath())){
+        valuesChecked.put(vv.getField().getAsSchemaPath().getRootSegment().getPath(), 0);
         columnValCounter = 0;
       } else {
-        columnValCounter = valuesChecked.get(vv.getField().getName());
+        columnValCounter = valuesChecked.get(vv.getField().getAsSchemaPath().getRootSegment().getPath());
       }
       for (int j = 0; j < vv.getAccessor().getValueCount(); j++) {
         if (ParquetRecordReaderTest.VERBOSE_DEBUG){
@@ -138,8 +141,8 @@ public class ParquetResultListener implements UserResultsListener {
       if (ParquetRecordReaderTest.VERBOSE_DEBUG){
         System.out.println("\n" + vv.getAccessor().getValueCount());
       }
-      valuesChecked.remove(vv.getField().getName());
-      valuesChecked.put(vv.getField().getName(), columnValCounter);
+      valuesChecked.remove(vv.getField().getAsSchemaPath().getRootSegment().getPath());
+      valuesChecked.put(vv.getField().getAsSchemaPath().getRootSegment().getPath(), columnValCounter);
     }
 
     if (ParquetRecordReaderTest.VERBOSE_DEBUG){
@@ -149,7 +152,7 @@ public class ParquetResultListener implements UserResultsListener {
           System.out.println();
           for (VectorWrapper vw : batchLoader) {
             ValueVector v = vw.getValueVector();
-            System.out.print(Strings.padStart(v.getField().getName(), 20, ' ') + " ");
+            System.out.print(Strings.padStart(v.getField().getAsSchemaPath().getRootSegment().getPath(), 20, ' ') + " ");
 
           }
           System.out.println();
@@ -185,7 +188,10 @@ public class ParquetResultListener implements UserResultsListener {
       }
 
       assert valuesChecked.keySet().size() > 0;
+      result.release();
       future.set(null);
+    }else{
+      result.release();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
index 48c0773..4d9b606 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/parquet/TestParquetPhysicalPlan.java
@@ -65,7 +65,7 @@ public class TestParquetPhysicalPlan {
         count += b.getHeader().getRowCount();
         loader.load(b.getHeader().getDef(), b.getData());
         for (VectorWrapper vw : loader) {
-          System.out.print(vw.getValueVector().getField().getName() + ": ");
+          System.out.print(vw.getValueVector().getField().toExpr() + ": ");
           ValueVector vv = vw.getValueVector();
           for (int i = 0; i < vv.getAccessor().getValueCount(); i++) {
             Object o = vv.getAccessor().getObject(i);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
index cf5e128..d8c280a 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/work/batch/TestSpoolingBuffer.java
@@ -54,6 +54,7 @@ public class TestSpoolingBuffer {
       for(QueryResultBatch b : results) {
         if (b.getHeader().getRowCount() != 0)
           count += b.getHeader().getRowCount();
+        b.release();
       }
       assertEquals(500024, count);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/agg/hashagg/q6.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q6.json b/exec/java-exec/src/test/resources/agg/hashagg/q6.json
index 96f62a3..10f4cba 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q6.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q6.json
@@ -24,10 +24,10 @@
     pop : "project",
     @id : 2,
     exprs : [ {
-      ref : "output.$f0",
+      ref : "$f0",
       expr : "N_REGIONKEY"
     }, {
-      ref : "output.$f1",
+      ref : "$f1",
       expr : "N_NATIONKEY"
     } ],
     child : 1

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json b/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
index f224677..256bec6 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q7_1.json
@@ -24,13 +24,13 @@
     pop : "project",
     @id : 2,
     exprs : [ {
-      ref : "output.$f0",
+      ref : "$f0",
       expr : "N_REGIONKEY"
     }, {
-      ref : "output.$f1",
+      ref : "$f1",
       expr : "N_NATIONKEY"
     }, {
-      ref : "output.$f2",
+      ref : "$f2",
       expr : "N_NAME"
     } ],
     child : 1

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json b/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
index c03868d..d444fc8 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q7_2.json
@@ -24,13 +24,13 @@
     pop : "project",
     @id : 2,
     exprs : [ {
-      ref : "output.$f0",
+      ref : "$f0",
       expr : "N_REGIONKEY"
     }, {
-      ref : "output.$f1",
+      ref : "$f1",
       expr : "N_NAME"
     }, {
-      ref : "output.$f2",
+      ref : "$f2",
       expr : "N_NATIONKEY"
     } ],
     child : 1

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/agg/hashagg/q8.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q8.json b/exec/java-exec/src/test/resources/agg/hashagg/q8.json
index e097c22..a457aa9 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q8.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q8.json
@@ -26,10 +26,10 @@
     pop : "project",
     @id : 2,
     exprs : [ {
-      ref : "output.$f0",
+      ref : "$f0",
       expr : "O_CUSTKEY"
     }, {
-      ref : "output.$f1",
+      ref : "$f1",
       expr : "O_ORDERKEY"
     } ],
     child : 1

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json b/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
index 7f2ae12..adc5e75 100644
--- a/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
+++ b/exec/java-exec/src/test/resources/agg/hashagg/q8_1.json
@@ -24,10 +24,10 @@
     pop : "project",
     @id : 2,
     exprs : [ {
-      ref : "output.$f0",
+      ref : "$f0",
       expr : "O_CUSTKEY"
     }, {
-      ref : "output.$f1",
+      ref : "$f1",
       expr : "O_ORDERKEY"
     } ],
     child : 1

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastBigInt.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastBigInt.json b/exec/java-exec/src/test/resources/functions/cast/testCastBigInt.json
index 9033c0a..c0a1565 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastBigInt.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastBigInt.json
@@ -31,7 +31,7 @@
 		{ ref: "float8_cast", expr:"cast(float8col as bigint)" },
 		{ ref: "intcol", expr:"intcol"  },
 		{ ref: "int_cast", expr:"cast(intcol as bigint)" },
-		{ ref: "varchar_cast", expr:"cast(\"1256\" as bigint)" }
+		{ ref: "varchar_cast", expr:"cast('1256' as bigint)" }
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastFloat4.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastFloat4.json b/exec/java-exec/src/test/resources/functions/cast/testCastFloat4.json
index 3b8fa99..f5f6e2c 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastFloat4.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastFloat4.json
@@ -31,8 +31,8 @@
 		{ ref: "int_cast", expr:"cast(intcol as float4)" },
 		{ ref: "bigintcol", expr:"bigintcol"  },
 		{ ref: "bigint_cast", expr:"cast(bigintcol as float4)" },
-		{ ref: "varchar_cast1", expr:"cast(\"1256\" as float4)" },
-		{ ref: "varchar_cast2", expr:"cast(\"12.56\" as float4)" }
+		{ ref: "varchar_cast1", expr:"cast('1256' as float4)" },
+		{ ref: "varchar_cast2", expr:"cast('12.56' as float4)" }
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastFloat8.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastFloat8.json b/exec/java-exec/src/test/resources/functions/cast/testCastFloat8.json
index b1a2a20..c700dd4 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastFloat8.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastFloat8.json
@@ -31,8 +31,8 @@
 		{ ref: "int_cast", expr:"cast(intcol as float8)" },
 		{ ref: "bigintcol", expr:"bigintcol"  },
 		{ ref: "bigint_cast", expr:"cast(bigintcol as float8)" },
-		{ ref: "varchar_cast1", expr:"cast(\"1256\" as float8)" },
-		{ ref: "varchar_cast2", expr:"cast(\"12.56\" as float8)" }
+		{ ref: "varchar_cast1", expr:"cast('1256' as float8)" },
+		{ ref: "varchar_cast2", expr:"cast('12.56' as float8)" }
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastInt.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastInt.json b/exec/java-exec/src/test/resources/functions/cast/testCastInt.json
index a34a223..271e276 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastInt.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastInt.json
@@ -31,7 +31,7 @@
 		{ ref: "float8_cast", expr:"cast(float8col as int)" },
 		{ ref: "bigintcol", expr:"bigintcol"  },
 		{ ref: "bigint_cast", expr:"cast(bigintcol as int)" },
-		{ ref: "varchar_cast", expr:"cast(\"1256\" as int)" }
+		{ ref: "varchar_cast", expr:"cast('1256' as int)" }
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastNested.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastNested.json b/exec/java-exec/src/test/resources/functions/cast/testCastNested.json
index eb51e86..4a2d80a 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastNested.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastNested.json
@@ -31,7 +31,7 @@
 		{ ref: "int_float4_add", expr:"cast(intcol as bigint) + cast(float4col as bigint)" },
 		{ ref: "bigintcol", expr:"bigintcol" },
 		{ ref: "bigint2varchar2int", expr:"cast(cast(bigintcol as varchar(8)) as int)" },
-                { ref: "add_cast", expr:"cast(\"100\" as int) + cast(cast(200 as varchar(8)) as int)"}
+                { ref: "add_cast", expr:"cast('100' as int) + cast(cast(200 as varchar(8)) as int)"}
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testCastNumException.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testCastNumException.json b/exec/java-exec/src/test/resources/functions/cast/testCastNumException.json
index 71d6a6e..27103af 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testCastNumException.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testCastNumException.json
@@ -25,7 +25,7 @@
             child: 1,
             pop:"project",
             exprs: [
-		{ ref: "varchar_cast", expr:"cast(\"abc\" as int)" }
+		{ ref: "varchar_cast", expr:"cast('abc' as int)" }
             ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/cast/testICastConstant.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/cast/testICastConstant.json b/exec/java-exec/src/test/resources/functions/cast/testICastConstant.json
index 22b868c..d85dd6c 100644
--- a/exec/java-exec/src/test/resources/functions/cast/testICastConstant.json
+++ b/exec/java-exec/src/test/resources/functions/cast/testICastConstant.json
@@ -24,25 +24,25 @@
             exprs: [
 			{ ref: "BigIntAddFloat8", expr:"10+20.1" },
 			{ ref: "Float8AddBigInt", expr:"20.1+10" },
-			{ ref: "Float8AddChar", expr:"20.1 + \"10\"" },
-		        { ref: "IntAddFloat8", expr:"cast(\"10\" as int) + 20.1"},
-		        { ref: "IntAddBigInt", expr:"cast(\"10\" as int) + cast(\"20\" as bigint)"},
-		        { ref: "BigIntAddInt", expr:"cast(\"10\" as bigint) + cast(\"20\" as int)"},
-		        { ref: "IntAddFloat8", expr:"cast(\"10\" as int) + cast(\"20.1\" as float8)"},
-		        { ref: "Float8AddInt", expr:"cast(\"20.1\" as float8) + cast(\"10\" as int) "},
-		        { ref: "IntAddFloat4", expr:"cast(\"10\" as int) + cast(\"20.1\" as float4)"},
-		        { ref: "BigIntAddFloat4", expr:"cast(\"10\" as bigint) + cast(\"20.1\" as float4)"},
-		        { ref: "BigIntAddFloat8", expr:"cast(\"10\" as bigint) + cast(\"20.1\" as float8)"},
-		        { ref: "Float4AddFloat8", expr:"cast(\"10\" as float4) + cast(\"20.1\" as float8)"},
-		        { ref: "CharAddFloat4", expr:"\"10\" + cast(\"20.1\" as float4)"},
-		        { ref: "CharAddFloat8", expr:"\"10\" + cast(\"20.1\" as float8)"},
-		        { ref: "Float4AddFloat8", expr:"cast(\"10\" as float4) + \"20.1\" "},
-		        { ref: "Float8AddChar", expr:"cast(\"10\" as float8) + \"20.1\" "},
+			{ ref: "Float8AddChar", expr:"20.1 + '10'" },
+		        { ref: "IntAddFloat8", expr:"cast('10' as int) + 20.1"},
+		        { ref: "IntAddBigInt", expr:"cast('10' as int) + cast('20' as bigint)"},
+		        { ref: "BigIntAddInt", expr:"cast('10' as bigint) + cast('20' as int)"},
+		        { ref: "IntAddFloat8", expr:"cast('10' as int) + cast('20.1' as float8)"},
+		        { ref: "Float8AddInt", expr:"cast('20.1' as float8) + cast('10' as int) "},
+		        { ref: "IntAddFloat4", expr:"cast('10' as int) + cast('20.1' as float4)"},
+		        { ref: "BigIntAddFloat4", expr:"cast('10' as bigint) + cast('20.1' as float4)"},
+		        { ref: "BigIntAddFloat8", expr:"cast('10' as bigint) + cast('20.1' as float8)"},
+		        { ref: "Float4AddFloat8", expr:"cast('10' as float4) + cast('20.1' as float8)"},
+		        { ref: "CharAddFloat4", expr:"'10' + cast('20.1' as float4)"},
+		        { ref: "CharAddFloat8", expr:"'10' + cast('20.1' as float8)"},
+		        { ref: "Float4AddFloat8", expr:"cast('10' as float4) + '20.1' "},
+		        { ref: "Float8AddChar", expr:"cast('10' as float8) + '20.1' "},
 			{ ref: "CompBigIntFloat8", expr:"10 < 20.1" },
-		        { ref: "CompCharFloat8", expr:"\"10\" < 20.1" },
-		        { ref: "CompFloat8Char", expr:"20.1 > \"10\" " },
-		        { ref: "nested1", expr:" 20.1  + 10 > \"10\" " },
-		        { ref: "nested2", expr:" 20.1  + 10 > \"10\"  + 15.1" }
+		        { ref: "CompCharFloat8", expr:"'10' < 20.1" },
+		        { ref: "CompFloat8Char", expr:"20.1 > '10' " },
+		        { ref: "nested1", expr:" 20.1  + 10 > '10' " },
+		        { ref: "nested2", expr:" 20.1  + 10 > '10'  + 15.1" }
 		   ]	
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/simple_math_functions.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/simple_math_functions.json b/exec/java-exec/src/test/resources/functions/simple_math_functions.json
index 28ac5b2..dde41cd 100644
--- a/exec/java-exec/src/test/resources/functions/simple_math_functions.json
+++ b/exec/java-exec/src/test/resources/functions/simple_math_functions.json
@@ -21,10 +21,10 @@
     pop : "project",
     @id : 2,
     exprs : [
-    { ref : "output.INTMUL", expr : " 1 * 2 "},
-    { ref : "output.FLOATMUL", expr : " 1.1 * 2.2 "},
-    { ref : "output.INTADD", expr : " 1 + 2 " },
-    { ref : "output.FLOATADD", expr : " 1.1 + 2.2 " } ],
+    { ref : "INTMUL", expr : " 1 * 2 "},
+    { ref : "FLOATMUL", expr : " 1.1 * 2.2 "},
+    { ref : "INTADD", expr : " 1 + 2 " },
+    { ref : "FLOATADD", expr : " 1.1 + 2.2 " } ],
     child : 1
   }, {
     pop : "screen",

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testCharLength.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testCharLength.json b/exec/java-exec/src/test/resources/functions/string/testCharLength.json
index 00d1a3b..ae29f67 100644
--- a/exec/java-exec/src/test/resources/functions/string/testCharLength.json
+++ b/exec/java-exec/src/test/resources/functions/string/testCharLength.json
@@ -23,16 +23,16 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "char_length(\"aababcdf\")"},
-						  { ref: "col2", expr: "char_length(\"\")"},
+						  { ref: "col1", expr: "char_length('aababcdf')"},
+						  { ref: "col2", expr: "char_length('')"},
 						  { ref: "col3", expr: "char_length(varcharcol)"},
 						  { ref: "col4", expr: "char_length(nullvarcharcol)"},
-						  { ref: "col5", expr: "character_length(\"aababcdf\")"},
-						  { ref: "col6", expr: "character_length(\"\")"},
+						  { ref: "col5", expr: "character_length('aababcdf')"},
+						  { ref: "col6", expr: "character_length('')"},
   					  { ref: "col7", expr: "character_length(varcharcol)"},
 						  { ref: "col8", expr: "character_length(nullvarcharcol)"},
-						  { ref: "col9", expr: "length(\"aababcdf\")"},
-						  { ref: "col10", expr: "length(\"\")"},
+						  { ref: "col9", expr: "length('aababcdf')"},
+						  { ref: "col10", expr: "length('')"},
   					  { ref: "col11", expr: "length(varcharcol)"},
 						  { ref: "col12", expr: "length(nullvarcharcol)"}
           ]

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testConcat.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testConcat.json b/exec/java-exec/src/test/resources/functions/string/testConcat.json
index ccefdd0..5b21710 100644
--- a/exec/java-exec/src/test/resources/functions/string/testConcat.json
+++ b/exec/java-exec/src/test/resources/functions/string/testConcat.json
@@ -23,10 +23,10 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "col1", expr: "concat(\"abc\", \"ABC\")"}, 
-              { ref: "col2", expr: "concat(\"abc\", \"\")"}, 
-              { ref: "col3", expr: "concat(\"\", \"ABC\")"}, 
-              { ref: "col4", expr: "concat(\"\", \"\")"} 
+              { ref: "col1", expr: "concat('abc', 'ABC')"}, 
+              { ref: "col2", expr: "concat('abc', '')"}, 
+              { ref: "col3", expr: "concat('', 'ABC')"}, 
+              { ref: "col4", expr: "concat('', '')"} 
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testLeft.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testLeft.json b/exec/java-exec/src/test/resources/functions/string/testLeft.json
index e043df5..e8126a5 100644
--- a/exec/java-exec/src/test/resources/functions/string/testLeft.json
+++ b/exec/java-exec/src/test/resources/functions/string/testLeft.json
@@ -23,13 +23,13 @@
             child: 1,
             pop:"project",
             exprs: [
-							{ ref: "col1", expr: "left(\"abcdef\", 2)"},
-							{ ref: "col2", expr: "left(\"abcdef\", 6)"}, 
-              { ref: "col3", expr: "left(\"abcdef\", 7)"},
-              { ref: "col4", expr: "left(\"abcdef\", -2)"},
-              { ref: "col5", expr: "left(\"abcdef\", -5)"},
-              { ref: "col6", expr: "left(\"abcdef\", -6)"},
-              { ref: "col7", expr: "left(\"abcdef\", -7)"}
+							{ ref: "col1", expr: "left('abcdef', 2)"},
+							{ ref: "col2", expr: "left('abcdef', 6)"}, 
+              { ref: "col3", expr: "left('abcdef', 7)"},
+              { ref: "col4", expr: "left('abcdef', -2)"},
+              { ref: "col5", expr: "left('abcdef', -5)"},
+              { ref: "col6", expr: "left('abcdef', -6)"},
+              { ref: "col7", expr: "left('abcdef', -7)"}
      			]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testLike.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testLike.json b/exec/java-exec/src/test/resources/functions/string/testLike.json
index 92b501c..fc99ef2 100644
--- a/exec/java-exec/src/test/resources/functions/string/testLike.json
+++ b/exec/java-exec/src/test/resources/functions/string/testLike.json
@@ -23,10 +23,10 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "islike", expr:" like(\"abc\", \"abc\")" },
-              { ref: "islike", expr:" like(\"abc\", \"a%\")" },
-              { ref: "islike", expr:" like(\"abc\", \"_b_\")" },
-              { ref: "islike", expr:" like(\"abc\", \"c\")" }
+              { ref: "islike", expr:" like('abc', 'abc')" },
+              { ref: "islike", expr:" like('abc', 'a%')" },
+              { ref: "islike", expr:" like('abc', '_b_')" },
+              { ref: "islike", expr:" like('abc', 'c')" }
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testLower.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testLower.json b/exec/java-exec/src/test/resources/functions/string/testLower.json
index 5e10920..d6351d8 100644
--- a/exec/java-exec/src/test/resources/functions/string/testLower.json
+++ b/exec/java-exec/src/test/resources/functions/string/testLower.json
@@ -23,9 +23,9 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "lower", expr: "lower(\"ABcEFgh\")"}, 
-              { ref: "lower", expr: "lower(\"aBc\")"}, 
-              { ref: "lower", expr: "lower(\"\")"}
+              { ref: "lower", expr: "lower('ABcEFgh')"}, 
+              { ref: "lower", expr: "lower('aBc')"}, 
+              { ref: "lower", expr: "lower('')"}
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testLpad.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testLpad.json b/exec/java-exec/src/test/resources/functions/string/testLpad.json
index 071acfe..017df71 100644
--- a/exec/java-exec/src/test/resources/functions/string/testLpad.json
+++ b/exec/java-exec/src/test/resources/functions/string/testLpad.json
@@ -23,16 +23,16 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "lpad(\"abcdef\", 0, \"abc\")"},
-						  { ref: "col2", expr: "lpad(\"abcdef\", -3, \"abc\")"},
-						  { ref: "col3", expr: "lpad(\"abcdef\", 6, \"abc\")"},
-						  { ref: "col4", expr: "lpad(\"abcdef\", 2, \"abc\")"},
-						  { ref: "col5", expr: "lpad(\"abcdef\", 2, \"\")"},
-						  { ref: "col7", expr: "lpad(\"abcdef\", 10, \"\")"},
-						  { ref: "col8", expr: "lpad(\"abcdef\", 10, \"A\")"},
-						  { ref: "col9", expr: "lpad(\"abcdef\", 10, \"AB\")"},
-						  { ref: "col10", expr: "lpad(\"abcdef\", 10, \"ABC\")"},
-						  { ref: "col11", expr: "lpad(\"abcdef\", 10, \"ABCDEFGHIJKLMN\")"}
+						  { ref: "col1", expr: "lpad('abcdef', 0, 'abc')"},
+						  { ref: "col2", expr: "lpad('abcdef', -3, 'abc')"},
+						  { ref: "col3", expr: "lpad('abcdef', 6, 'abc')"},
+						  { ref: "col4", expr: "lpad('abcdef', 2, 'abc')"},
+						  { ref: "col5", expr: "lpad('abcdef', 2, '')"},
+						  { ref: "col7", expr: "lpad('abcdef', 10, '')"},
+						  { ref: "col8", expr: "lpad('abcdef', 10, 'A')"},
+						  { ref: "col9", expr: "lpad('abcdef', 10, 'AB')"},
+						  { ref: "col10", expr: "lpad('abcdef', 10, 'ABC')"},
+						  { ref: "col11", expr: "lpad('abcdef', 10, 'ABCDEFGHIJKLMN')"}
 
         ]
         },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testLtrim.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testLtrim.json b/exec/java-exec/src/test/resources/functions/string/testLtrim.json
index e1f1a68..f801236 100644
--- a/exec/java-exec/src/test/resources/functions/string/testLtrim.json
+++ b/exec/java-exec/src/test/resources/functions/string/testLtrim.json
@@ -23,12 +23,12 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "ltrim(\"abcdef\", \"abc\")"},
-						  { ref: "col2", expr: "ltrim(\"abcdef\", \"\")"},
-						  { ref: "col3", expr: "ltrim(\"abcdabc\", \"abc\")"},
-						  { ref: "col4", expr: "ltrim(\"abc\", \"abc\")"},
-						  { ref: "col5", expr: "ltrim(\"\", \"abc\")"},
-						  { ref: "col6", expr: "ltrim(\"\", \"\")"}
+						  { ref: "col1", expr: "ltrim('abcdef', 'abc')"},
+						  { ref: "col2", expr: "ltrim('abcdef', '')"},
+						  { ref: "col3", expr: "ltrim('abcdabc', 'abc')"},
+						  { ref: "col4", expr: "ltrim('abc', 'abc')"},
+						  { ref: "col5", expr: "ltrim('', 'abc')"},
+						  { ref: "col6", expr: "ltrim('', '')"}
         ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testPosition.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testPosition.json b/exec/java-exec/src/test/resources/functions/string/testPosition.json
index 8abeeae..d9cccf9 100644
--- a/exec/java-exec/src/test/resources/functions/string/testPosition.json
+++ b/exec/java-exec/src/test/resources/functions/string/testPosition.json
@@ -23,14 +23,14 @@
             child: 1,
             pop:"project",
             exprs: [
-              { ref: "col1", expr: "position(\"abc\", \"AabcabcB\")"},
-              { ref: "col2", expr: "position(\"\", \"AabcabcB\")"},
-              { ref: "col3", expr: "position(\"abc\", \"\")"},
-              { ref: "col4", expr: "position(\"\", \"\")"},
-              { ref: "col5", expr: "strpos(\"AabcabcB\", \"abc\")" },
-              { ref: "col6", expr: "strpos(\"\", \"AabcabcB\")"},
-              { ref: "col7", expr: "strpos(\"\", \"abc\")"},
-              { ref: "col8", expr: "strpos(\"\", \"\")"}
+              { ref: "col1", expr: "position('abc', 'AabcabcB')"},
+              { ref: "col2", expr: "position('', 'AabcabcB')"},
+              { ref: "col3", expr: "position('abc', '')"},
+              { ref: "col4", expr: "position('', '')"},
+              { ref: "col5", expr: "strpos('AabcabcB', 'abc')" },
+              { ref: "col6", expr: "strpos('', 'AabcabcB')"},
+              { ref: "col7", expr: "strpos('', 'abc')"},
+              { ref: "col8", expr: "strpos('', '')"}
            ]
         },
         {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testRegexpReplace.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testRegexpReplace.json b/exec/java-exec/src/test/resources/functions/string/testRegexpReplace.json
index 3e3874a..268ec00 100644
--- a/exec/java-exec/src/test/resources/functions/string/testRegexpReplace.json
+++ b/exec/java-exec/src/test/resources/functions/string/testRegexpReplace.json
@@ -23,9 +23,9 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "regexp_replace(\"Thomas\", \".[mN]a.\", \"M\")" },
-						  { ref: "col1", expr: "regexp_replace(\"Thomas\", \".[mN]a.\", \"\")" },
-						  { ref: "col1", expr: "regexp_replace(\"Thomas\", \"ef\", \"AB\")" }
+						  { ref: "col1", expr: "regexp_replace('Thomas', '.[mN]a.', 'M')" },
+						  { ref: "col1", expr: "regexp_replace('Thomas', '.[mN]a.', '')" },
+						  { ref: "col1", expr: "regexp_replace('Thomas', 'ef', 'AB')" }
 
       ]
         },

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/test/resources/functions/string/testReplace.json
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/functions/string/testReplace.json b/exec/java-exec/src/test/resources/functions/string/testReplace.json
index 4041dd4..8cf8c64 100644
--- a/exec/java-exec/src/test/resources/functions/string/testReplace.json
+++ b/exec/java-exec/src/test/resources/functions/string/testReplace.json
@@ -23,12 +23,12 @@
             child: 1,
             pop:"project",
             exprs: [
-						  { ref: "col1", expr: "replace(\"aababcdf\", \"ab\", \"AB\")"},
-						  { ref: "col2", expr: "replace(\"aababcdf\", \"a\", \"AB\")"},
-						  { ref: "col3", expr: "replace(\"aababcdf\", \"\", \"AB\")"},
-						  { ref: "col4", expr: "replace(\"aababcdf\", \"ab\", \"\")"},
-						  { ref: "col5", expr: "replace(\"abc\", \"abc\", \"ABCD\")"},
-						  { ref: "col6", expr: "replace(\"abc\", \"abcdefg\", \"ABCD\")"}
+						  { ref: "col1", expr: "replace('aababcdf', 'ab', 'AB')"},
+						  { ref: "col2", expr: "replace('aababcdf', 'a', 'AB')"},
+						  { ref: "col3", expr: "replace('aababcdf', '', 'AB')"},
+						  { ref: "col4", expr: "replace('aababcdf', 'ab', '')"},
+						  { ref: "col5", expr: "replace('abc', 'abc', 'ABCD')"},
+						  { ref: "col6", expr: "replace('abc', 'abcdefg', 'ABCD')"}
           ]
         },
         {


[22/51] [abbrv] git commit: not function implementation

Posted by ja...@apache.org.
not function implementation


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

Branch: refs/heads/master
Commit: ca58de14810c7556c507dd1013aa488f0cafb6bb
Parents: 8e2b950
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 15:02:01 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../org/apache/drill/exec/expr/fn/impl/Not.java | 57 ++++++++++++++++++++
 1 file changed, 57 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ca58de14/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Not.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Not.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Not.java
new file mode 100644
index 0000000..db33247
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/Not.java
@@ -0,0 +1,57 @@
+/**
+ * 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.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.BitHolder;
+import org.apache.drill.exec.expr.holders.NullableBitHolder;
+import org.apache.drill.exec.record.RecordBatch;
+
+public class Not {
+
+  @FunctionTemplate(names = {"not"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class Optional implements DrillSimpleFunc {
+
+    @Param NullableBitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = (in.value == 0 ? 1 : 0);
+    }
+  }
+
+  @FunctionTemplate(names = {"not"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = FunctionTemplate.NullHandling.INTERNAL)
+  public static class Required implements DrillSimpleFunc {
+
+    @Param BitHolder in;
+    @Output BitHolder out;
+
+    public void setup(RecordBatch incoming) { }
+
+    public void eval() {
+      out.value = in.value == 0 ? 1 : 0;
+    }
+  }
+}
\ No newline at end of file


[25/51] [abbrv] git commit: Bug in DrillOptiq while generating cast functions to Float, Double, IntervalYear and IntervalDay

Posted by ja...@apache.org.
Bug in DrillOptiq while generating cast functions to Float, Double, IntervalYear and IntervalDay


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

Branch: refs/heads/master
Commit: f2f889c1dab3ab256e9b38914b12c91fac5146e2
Parents: 443e4e9
Author: Mehant Baid <me...@gmail.com>
Authored: Tue Apr 1 11:18:34 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/ExpressionStringBuilder.java     | 4 ++--
 .../org/apache/drill/exec/planner/logical/DrillOptiq.java    | 8 ++++----
 2 files changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f2f889c1/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 16b3f4d..787e4af 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -139,7 +139,7 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
   public Void visitIntervalYearConstant(IntervalYearExpression lExpr, StringBuilder sb) throws RuntimeException {
     sb.append("cast( \"");
     sb.append(Period.years(lExpr.getIntervalYear()).toString());
-    sb.append("\" as INTERVAL)");
+    sb.append("\" as INTERVALYEAR)");
     return null;
   }
 
@@ -147,7 +147,7 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
   public Void visitIntervalDayConstant(IntervalDayExpression lExpr, StringBuilder sb) throws RuntimeException {
     sb.append("cast( \"");
     sb.append(Period.days(lExpr.getIntervalDay()).plusMillis(lExpr.getIntervalMillis()).toString());
-    sb.append("\" as INTERVAL)");
+    sb.append("\" as INTERVALDAY)");
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/f2f889c1/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 1f8e14b..4926d7b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -199,11 +199,11 @@ public class DrillOptiq {
           break;
       
         case "INTEGER": castType = Types.required(MinorType.INT); break;
-        case "FLOAT": Types.required(MinorType.FLOAT4); break;
-        case "DOUBLE": Types.required(MinorType.FLOAT8); break;
+        case "FLOAT": castType = Types.required(MinorType.FLOAT4); break;
+        case "DOUBLE": castType = Types.required(MinorType.FLOAT8); break;
         case "DECIMAL": throw new UnsupportedOperationException("Need to add decimal.");
-        case "INTERVAL_YEAR_MONTH": Types.required(MinorType.INTERVALYEAR); break;
-        case "INTERVAL_DAY_TIME": Types.required(MinorType.INTERVALDAY); break;
+        case "INTERVAL_YEAR_MONTH": castType = Types.required(MinorType.INTERVALYEAR); break;
+        case "INTERVAL_DAY_TIME": castType = Types.required(MinorType.INTERVALDAY); break;
         default: castType = Types.required(MinorType.valueOf(call.getType().getSqlTypeName().getName()));
       }
       


[13/51] [abbrv] git commit: fix merging receiver

Posted by ja...@apache.org.
fix merging receiver


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

Branch: refs/heads/master
Commit: bf3fa6607290a4d5a976bf48a16927690a981da8
Parents: 02b2068
Author: Steven Phillips <sp...@maprtech.com>
Authored: Mon Mar 31 16:33:05 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../exec/physical/impl/mergereceiver/MergingRecordBatch.java | 8 +++++---
 1 file changed, 5 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/bf3fa660/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index 13ed4c9..ee2244e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -139,8 +139,9 @@ public class MergingRecordBatch implements RecordBatch {
           context.fail(e);
           return IterOutcome.STOP;
         }
-        if (rawBatch.getHeader().getDef().getRecordCount() != 0)
+        if (rawBatch.getHeader().getDef().getRecordCount() != 0) {
           rawBatches.add(rawBatch);
+        }
       }
 
       // allocate the incoming record batch loaders
@@ -179,7 +180,7 @@ public class MergingRecordBatch implements RecordBatch {
 
         // allocate a new value vector
         ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), context.getAllocator());
-        VectorAllocator allocator = VectorAllocator.getAllocator(v.getValueVector(), outgoingVector);
+        VectorAllocator allocator = VectorAllocator.getAllocator(outgoingVector, 50);
         allocator.alloc(DEFAULT_ALLOC_RECORD_COUNT);
         allocators.add(allocator);
         outgoingContainer.add(outgoingVector);
@@ -371,6 +372,7 @@ public class MergingRecordBatch implements RecordBatch {
     final ClassGenerator<MergingReceiverGeneratorBase> cg =
         CodeGenerator.getRoot(MergingReceiverGeneratorBase.TEMPLATE_DEFINITION, context.getFunctionRegistry());
     JExpression inIndex = JExpr.direct("inIndex");
+    JExpression outIndex = JExpr.direct("outIndex");
 
     JType valueVector2DArray = cg.getModel().ref(ValueVector.class).array().array();
     JType valueVectorArray = cg.getModel().ref(ValueVector.class).array();
@@ -587,7 +589,7 @@ public class MergingRecordBatch implements RecordBatch {
         ((JExpression) JExpr.cast(vvClass, outgoingVectors.component(JExpr.lit(fieldIdx))))
           .invoke("copyFrom")
           .arg(inIndex)
-          .arg(outgoingBatch.invoke("getRecordCount"))
+          .arg(outIndex)
           .arg(JExpr.cast(vvClass,
                           ((JExpression) incomingVectors.component(JExpr.direct("inBatch")))
                             .component(JExpr.lit(fieldIdx)))));


[02/51] [abbrv] git commit: Start on move towards optiq for I --> p

Posted by ja...@apache.org.
Start on move towards optiq for I --> p

optiq-opt1 More WIP.


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

Branch: refs/heads/master
Commit: 22c4190715d081bf5c4c684c5d67131e973c59f2
Parents: 71e912a
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Feb 23 07:51:01 2014 -0800
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:08 2014 -0700

----------------------------------------------------------------------
 .../apache/drill/common/logical/data/Scan.java  | 17 +----
 exec/java-exec/pom.xml                          |  6 ++
 .../apache/drill/exec/opt/BasicOptimizer.java   |  2 +-
 .../drill/exec/physical/base/GroupScan.java     |  3 +-
 .../exec/planner/common/BaseFilterRel.java      | 53 ++++++++++++++
 .../exec/planner/common/BaseProjectRel.java     | 68 ++++++++++++++++++
 .../drill/exec/planner/common/BaseScanRel.java  | 36 ++++++++++
 .../exec/planner/common/BaseScreenRel.java      | 41 +++++++++++
 .../exec/planner/common/DrillLimitRel.java      | 75 ++++++++++++++++++++
 .../exec/planner/common/DrillStoreRel.java      | 47 ++++++++++++
 .../exec/planner/common/DrillUnionRel.java      | 73 +++++++++++++++++++
 .../planner/logical/DrillAggregateRule.java     |  2 +-
 .../exec/planner/logical/DrillFilterRel.java    | 20 ++----
 .../exec/planner/logical/DrillFilterRule.java   |  7 +-
 .../exec/planner/logical/DrillJoinRule.java     |  2 +-
 .../exec/planner/logical/DrillLimitRel.java     |  2 +
 .../exec/planner/logical/DrillLimitRule.java    |  5 +-
 .../drill/exec/planner/logical/DrillOptiq.java  |  2 +-
 .../exec/planner/logical/DrillProjectRel.java   | 30 ++------
 .../exec/planner/logical/DrillProjectRule.java  |  2 +-
 .../drill/exec/planner/logical/DrillRel.java    |  2 +-
 .../exec/planner/logical/DrillRuleSets.java     | 58 ++++++++++-----
 .../exec/planner/logical/DrillScanRel.java      | 15 +---
 .../exec/planner/logical/DrillScanRule.java     |  6 +-
 .../exec/planner/logical/DrillScreenRel.java    | 33 +--------
 .../exec/planner/logical/DrillSortRule.java     |  4 +-
 .../exec/planner/logical/DrillStoreRel.java     |  2 +
 .../drill/exec/planner/logical/DrillTable.java  | 19 +++--
 .../exec/planner/logical/DrillUnionRel.java     |  2 +
 .../exec/planner/logical/DrillUnionRule.java    |  3 +-
 .../exec/planner/logical/DrillValuesRel.java    |  2 +-
 .../exec/planner/logical/DrillValuesRule.java   |  2 +-
 .../exec/planner/logical/DynamicDrillTable.java |  5 +-
 .../planner/logical/EnumerableDrillRule.java    | 11 +--
 .../drill/exec/planner/physical/HashPrel.java   | 37 ++++++++++
 .../planner/physical/PhysicalPlanCreator.java   | 20 ++++++
 .../drill/exec/planner/physical/Prel.java       | 17 +++++
 .../exec/planner/physical/ProjectPrel.java      | 38 ++++++++++
 .../exec/planner/physical/ProjectPrule.java     | 45 ++++++++++++
 .../drill/exec/planner/physical/ScanPrel.java   | 29 ++++++++
 .../drill/exec/planner/physical/ScanPrule.java  | 42 +++++++++++
 .../drill/exec/planner/physical/ScreenPrel.java | 25 +++++++
 .../drill/exec/planner/physical/SortPrel.java   | 36 ++++++++++
 .../exec/planner/physical/StorageRule.java      |  5 ++
 .../exec/planner/physical/StreamAggPrel.java    | 36 ++++++++++
 .../exec/planner/physical/TopNSortPrel.java     | 36 ++++++++++
 .../drill/exec/planner/sql/DrillSqlWorker.java  | 41 +++++++++--
 .../exec/planner/torel/ConversionContext.java   | 18 ++---
 .../drill/exec/store/AbstractStoragePlugin.java |  7 +-
 .../apache/drill/exec/store/StoragePlugin.java  |  6 +-
 .../drill/exec/store/dfs/FileSystemPlugin.java  | 18 +++--
 .../drill/exec/store/dfs/FormatPlugin.java      |  2 +-
 .../exec/store/dfs/WorkspaceSchemaFactory.java  |  8 ++-
 .../exec/store/dfs/easy/EasyFormatPlugin.java   |  8 +--
 .../exec/store/dfs/easy/EasyGroupScan.java      |  7 +-
 .../drill/exec/store/dfs/easy/EasySubScan.java  | 11 +--
 .../exec/store/easy/json/JSONFormatPlugin.java  |  4 +-
 .../exec/store/easy/json/JSONRecordReader.java  | 11 ++-
 .../exec/store/hive/HiveStoragePlugin.java      | 10 ++-
 .../exec/store/hive/schema/DrillHiveTable.java  |  5 +-
 .../store/hive/schema/HiveSchemaFactory.java    |  9 +--
 .../exec/store/mock/MockStorageEngine.java      | 16 +++--
 .../exec/store/parquet/ParquetFormatPlugin.java |  4 +-
 .../exec/store/parquet/ParquetGroupScan.java    | 14 +---
 .../exec/store/parquet/ParquetRecordReader.java | 21 +++---
 .../exec/store/parquet/ParquetRowGroupScan.java | 11 +--
 .../store/parquet/ParquetScanBatchCreator.java  |  1 -
 .../apache/drill/exec/work/foreman/Foreman.java | 17 -----
 .../planner/physical/TestPhysicalPlanning.java  | 39 ++++++++++
 .../exec/store/json/JSONRecordReaderTest.java   | 14 ++--
 .../src/test/resources/storage-engines.json     |  7 +-
 .../org/apache/drill/jdbc/test/JdbcTest.java    |  2 -
 72 files changed, 1049 insertions(+), 285 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Scan.java b/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
index c36c2d1..f7e69b3 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Scan.java
@@ -30,14 +30,12 @@ import com.fasterxml.jackson.annotation.JsonTypeName;
 public class Scan extends SourceOperator{
 	private final String storageEngine;
 	private final JSONOptions selection;
-	private final FieldReference outputReference;
 	
 	@JsonCreator
-  public Scan(@JsonProperty("storageengine") String storageEngine, @JsonProperty("selection") JSONOptions selection, @JsonProperty("ref") FieldReference outputReference) {
+  public Scan(@JsonProperty("storageengine") String storageEngine, @JsonProperty("selection") JSONOptions selection) {
     super();
     this.storageEngine = storageEngine;
     this.selection = selection;
-    this.outputReference = outputReference;
   }
 
   @JsonProperty("storageengine")
@@ -49,11 +47,6 @@ public class Scan extends SourceOperator{
     return selection;
   }
 
-  @JsonProperty("ref")
-  public FieldReference getOutputReference() {
-    return outputReference;
-  }
-
   @Override
   public <T, X, E extends Throwable> T accept(LogicalVisitor<T, X, E> logicalVisitor, X value) throws E {
       return logicalVisitor.visitScan(this, value);
@@ -67,7 +60,6 @@ public class Scan extends SourceOperator{
   public static class Builder extends AbstractBuilder<Scan>{
     private String storageEngine;
     private JSONOptions selection;
-    private FieldReference outputReference;
 
     public Builder storageEngine(String storageEngine) {
       this.storageEngine = storageEngine;
@@ -79,13 +71,8 @@ public class Scan extends SourceOperator{
       return this;
     }
 
-    public Builder outputReference(FieldReference outputReference) {
-      this.outputReference = outputReference;
-      return this;
-    }
-
     public Scan build() {
-      return new Scan(storageEngine, selection, outputReference);
+      return new Scan(storageEngine, selection);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/pom.xml
----------------------------------------------------------------------
diff --git a/exec/java-exec/pom.xml b/exec/java-exec/pom.xml
index 0b88441..e88fbec 100644
--- a/exec/java-exec/pom.xml
+++ b/exec/java-exec/pom.xml
@@ -87,6 +87,12 @@
       <scope>test</scope>
     </dependency>
     <dependency>
+      <groupId>pentaho</groupId>
+      <artifactId>mondrian-data-foodmart-json</artifactId>
+      <version>0.3.2</version>
+      <scope>test</scope>
+    </dependency>    
+    <dependency>
       <groupId>com.twitter</groupId>
       <artifactId>parquet-column</artifactId>
       <version>1.2.8</version>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
index 42de28b..36b8373 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/opt/BasicOptimizer.java
@@ -187,7 +187,7 @@ public class BasicOptimizer extends Optimizer{
       StoragePlugin engine;
       try {
         engine = context.getStorage().getEngine(config);
-        return engine.getPhysicalScan(scan);
+        return engine.getPhysicalScan(scan.getSelection());
       } catch (IOException | ExecutionSetupException e) {
         throw new OptimizerException("Failure while attempting to retrieve storage engine.", e);
       }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
index e51ebb6..314bc84 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/base/GroupScan.java
@@ -21,6 +21,7 @@ import java.util.List;
 
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.exceptions.PhysicalOperatorSetupException;
+import org.apache.drill.exec.planner.physical.Prel;
 import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 
 import com.fasterxml.jackson.annotation.JsonIgnore;
@@ -29,7 +30,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
  * A GroupScan operator represents all data which will be scanned by a given physical
  * plan.  It is the superset of all SubScans for the plan.
  */
-public interface GroupScan extends Scan, HasAffinity{
+public interface GroupScan extends Scan, HasAffinity, Prel{
 
   public abstract void applyAssignments(List<DrillbitEndpoint> endpoints) throws PhysicalOperatorSetupException;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
new file mode 100644
index 0000000..42c0ec5
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseFilterRel.java
@@ -0,0 +1,53 @@
+/**
+ * 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.common;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.Filter;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.FilterRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexNode;
+
+
+public abstract class BaseFilterRel extends FilterRelBase{
+  protected BaseFilterRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
+    super(cluster, traits, child, condition);
+    assert getConvention() == convention;
+  }
+  
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+  }
+
+  protected LogicalExpression getFilterExpression(DrillParseContext context){
+    return DrillOptiq.toDrill(context, getChild(), getCondition());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
new file mode 100644
index 0000000..9314a29
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseProjectRel.java
@@ -0,0 +1,68 @@
+/**
+ * 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.common;
+
+import java.util.List;
+
+import org.apache.drill.common.expression.FieldReference;
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.logical.data.NamedExpression;
+import org.apache.drill.exec.planner.logical.DrillOptiq;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.eigenbase.rel.ProjectRelBase;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.util.Pair;
+
+import com.google.common.collect.Lists;
+
+/**
+ * Project implemented in Drill.
+ */
+public class BaseProjectRel extends ProjectRelBase{
+  protected BaseProjectRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
+      RelDataType rowType) {
+    super(cluster, traits, child, exps, rowType, Flags.BOXED);
+    assert getConvention() == convention;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+  }
+
+  private List<Pair<RexNode, String>> projects() {
+    return Pair.zip(exps, getRowType().getFieldNames());
+  }
+
+  protected List<NamedExpression> getProjectExpressions(DrillParseContext context){
+    List<NamedExpression> expressions = Lists.newArrayList();
+    for (Pair<RexNode, String> pair : projects()) {
+      LogicalExpression expr = DrillOptiq.toDrill(context, getChild(), pair.left);
+      expressions.add(new NamedExpression(expr, new FieldReference("output." + pair.right)));
+    }
+    return expressions;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
new file mode 100644
index 0000000..69b47ab
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScanRel.java
@@ -0,0 +1,36 @@
+/**
+ * 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.common;
+
+import org.apache.drill.exec.planner.logical.DrillTable;
+import org.eigenbase.rel.TableAccessRelBase;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+
+public abstract class BaseScanRel extends TableAccessRelBase{
+  protected final DrillTable drillTable;
+
+  public BaseScanRel(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
+    super(cluster, traits, table);
+    this.drillTable = table.unwrap(DrillTable.class);
+    assert drillTable != null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
new file mode 100644
index 0000000..4c1a318
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/BaseScreenRel.java
@@ -0,0 +1,41 @@
+/**
+ * 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.common;
+
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+public abstract class BaseScreenRel extends SingleRel {
+
+  public BaseScreenRel(Convention convention, RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
+    super(cluster, traitSet, input);
+    assert input.getConvention() == convention;
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    return super.computeSelfCost(planner).multiplyBy(.1);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
new file mode 100644
index 0000000..cfaf350
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRel.java
@@ -0,0 +1,75 @@
+/**
+ * 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.common;
+
+import java.math.BigDecimal;
+import java.util.List;
+
+import org.apache.drill.common.logical.data.Limit;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+import org.eigenbase.sql.type.SqlTypeName;
+
+public class DrillLimitRel extends SingleRel implements DrillRel {
+  private RexNode offset;
+  private RexNode fetch;
+
+  public DrillLimitRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, RexNode offset, RexNode fetch) {
+    super(cluster, traitSet, child);
+    this.offset = offset;
+    this.fetch = fetch;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new DrillLimitRel(getCluster(), traitSet, sole(inputs), offset, fetch);
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    LogicalOperator inputOp = implementor.visitChild(this, 0, getChild());
+    
+    // First offset to include into results (inclusive). Null implies it is starting from offset 0
+    int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
+
+    // Last offset to stop including into results (exclusive), translating fetch row counts into an offset.
+    // Null value implies including entire remaining result set from first offset
+    Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
+    Limit limit = new Limit(first, last);
+    limit.setInput(inputOp);
+    return limit;
+  }
+  
+  public static DrillLimitRel convert(Limit limit, ConversionContext context) throws InvalidRelException{
+    RelNode input = context.toRel(limit.getInput());
+    RexNode first = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getFirst()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+    RexNode last = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getLast()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));
+    return new DrillLimitRel(context.getCluster(), context.getLogicalTraits(), input, first, last);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.java
new file mode 100644
index 0000000..ffbfa94
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillStoreRel.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.common;
+
+import java.util.List;
+
+import net.hydromatic.optiq.prepare.Prepare.CatalogReader;
+
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.TableModificationRelBase;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class DrillStoreRel extends TableModificationRelBase implements DrillRel{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillStoreRel.class);
+
+  protected DrillStoreRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table, CatalogReader catalogReader,
+      RelNode child, Operation operation, List<String> updateColumnList, boolean flattened) {
+    super(cluster, traits, table, catalogReader, child, operation, updateColumnList, flattened);
+    
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
new file mode 100644
index 0000000..2794560
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillUnionRel.java
@@ -0,0 +1,73 @@
+/**
+ * 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.common;
+
+import java.util.List;
+
+import net.hydromatic.linq4j.Ord;
+
+import org.apache.drill.common.logical.data.Limit;
+import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.common.logical.data.Union;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.torel.ConversionContext;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.UnionRelBase;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+
+/**
+ * Union implemented in Drill.
+ */
+public class DrillUnionRel extends UnionRelBase implements DrillRel {
+  /** Creates a DrillUnionRel. */
+  public DrillUnionRel(RelOptCluster cluster, RelTraitSet traits,
+      List<RelNode> inputs, boolean all) {
+    super(cluster, traits, inputs, all);
+  }
+
+  @Override
+  public DrillUnionRel copy(RelTraitSet traitSet, List<RelNode> inputs,
+      boolean all) {
+    return new DrillUnionRel(getCluster(), traitSet, inputs, all);
+  }
+
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    // divide cost by two to ensure cheaper than EnumerableDrillRel
+    return super.computeSelfCost(planner).multiplyBy(.5);
+  }
+
+  @Override
+  public LogicalOperator implement(DrillImplementor implementor) {
+    Union.Builder builder = Union.builder();
+    for (Ord<RelNode> input : Ord.zip(inputs)) {
+      builder.addInput(implementor.visitChild(this, input.i, input.e));
+    }
+    builder.setDistinct(!all);
+    return builder.build();
+  }
+  
+  public static DrillUnionRel convert(Union union, ConversionContext context) throws InvalidRelException{
+    throw new UnsupportedOperationException();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
index 77f1ba6..273237a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillAggregateRule.java
@@ -41,7 +41,7 @@ public class DrillAggregateRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final AggregateRel aggregate = (AggregateRel) call.rel(0);
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = aggregate.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = aggregate.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final RelNode convertedInput = convert(input, traits);
     try {
       call.transformTo(new DrillAggregateRel(aggregate.getCluster(), traits, convertedInput, aggregate.getGroupSet(),

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
index cc147e3..867def4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRel.java
@@ -21,23 +21,18 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Filter;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.common.BaseFilterRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.eigenbase.rel.FilterRelBase;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexNode;
 
-/**
- * Filter implemented in Drill.
- */
-public class DrillFilterRel extends FilterRelBase implements DrillRel {
+
+public class DrillFilterRel extends BaseFilterRel implements DrillRel {
   protected DrillFilterRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, RexNode condition) {
-    super(cluster, traits, child, condition);
-    assert getConvention() == CONVENTION;
+    super(DRILL_LOGICAL, cluster, traits, child, condition);
   }
 
   @Override
@@ -46,14 +41,9 @@ public class DrillFilterRel extends FilterRelBase implements DrillRel {
   }
 
   @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
-  }
-
-  @Override
   public LogicalOperator implement(DrillImplementor implementor) {
     final LogicalOperator input = implementor.visitChild(this, 0, getChild());
-    Filter f = new Filter(DrillOptiq.toDrill(implementor.getContext(), getChild(), getCondition()));
+    Filter f = new Filter(getFilterExpression(implementor.getContext()));
     f.setInput(input);
     return f;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
index d4fb239..7cdc6ec 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillFilterRule.java
@@ -19,7 +19,10 @@ package org.apache.drill.exec.planner.logical;
 
 import org.eigenbase.rel.FilterRel;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.relopt.*;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
 
 /**
  * Rule that converts a {@link org.eigenbase.rel.FilterRel} to a Drill "filter" operation.
@@ -35,7 +38,7 @@ public class DrillFilterRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final FilterRel filter = (FilterRel) call.rel(0);
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = filter.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = filter.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final RelNode convertedInput = convert(input, traits);
     call.transformTo(new DrillFilterRel(filter.getCluster(), traits, convertedInput, filter.getCondition()));
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
index f32fa59..2a895a5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillJoinRule.java
@@ -43,7 +43,7 @@ public class DrillJoinRule extends RelOptRule {
     final JoinRel join = (JoinRel) call.rel(0);
     final RelNode left = call.rel(1);
     final RelNode right = call.rel(2);
-    final RelTraitSet traits = join.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = join.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
 
     final RelNode convertedLeft = convert(left, traits);
     final RelNode convertedRight = convert(right, traits);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
index 54be052..137df72 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRel.java
@@ -22,6 +22,8 @@ import java.util.List;
 
 import org.apache.drill.common.logical.data.Limit;
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
index 85c594e..f1d3bae 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillLimitRule.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.drill.exec.planner.common.DrillLimitRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.SortRel;
 import org.eigenbase.relopt.Convention;
@@ -52,8 +53,8 @@ public class DrillLimitRule extends RelOptRule {
       input = incomingSort.copy(incomingTraits, input, incomingSort.getCollation(), null, null);
     }
 
-    RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.CONVENTION));
-    call.transformTo(new DrillLimitRel(incomingSort.getCluster(), incomingTraits.plus(DrillRel.CONVENTION), convertedInput, incomingSort.offset, incomingSort.fetch));
+    RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
+    call.transformTo(new DrillLimitRel(incomingSort.getCluster(), incomingTraits.plus(DrillRel.DRILL_LOGICAL), convertedInput, incomingSort.offset, incomingSort.fetch));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
index 917ab53..752f31d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillOptiq.java
@@ -59,7 +59,7 @@ public class DrillOptiq {
   /**
    * Converts a tree of {@link RexNode} operators into a scalar expression in Drill syntax.
    */
-  static LogicalExpression toDrill(DrillParseContext context, RelNode input, RexNode expr) {
+  public static LogicalExpression toDrill(DrillParseContext context, RelNode input, RexNode expr) {
     final RexToDrill visitor = new RexToDrill(context, input);
     return expr.accept(visitor);
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
index ee3a0f4..eb9086d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
@@ -18,22 +18,16 @@
 package org.apache.drill.exec.planner.logical;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-import org.apache.drill.common.expression.FieldReference;
-import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.common.logical.data.Project;
+import org.apache.drill.exec.planner.common.BaseProjectRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
-import org.eigenbase.rel.ProjectRelBase;
-import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.reltype.RelDataTypeField;
@@ -41,45 +35,31 @@ import org.eigenbase.reltype.RelDataTypeFieldImpl;
 import org.eigenbase.reltype.RelRecordType;
 import org.eigenbase.rex.RexNode;
 import org.eigenbase.sql.type.SqlTypeName;
-import org.eigenbase.util.Pair;
 
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
 import com.google.common.collect.Lists;
 
 /**
  * Project implemented in Drill.
  */
-public class DrillProjectRel extends ProjectRelBase implements DrillRel {
+public class DrillProjectRel extends BaseProjectRel implements DrillRel {
   protected DrillProjectRel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
       RelDataType rowType) {
-    super(cluster, traits, child, exps, rowType, Flags.BOXED);
-    assert getConvention() == CONVENTION;
+    super(DRILL_LOGICAL, cluster, traits, child, exps, rowType);
   }
 
-
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new DrillProjectRel(getCluster(), traitSet, sole(inputs), new ArrayList<RexNode>(exps), rowType);
   }
 
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(0.1);
-  }
-
-  private List<Pair<RexNode, String>> projects() {
-    return Pair.zip(exps, getRowType().getFieldNames());
-  }
 
   @Override
   public LogicalOperator implement(DrillImplementor implementor) {
     LogicalOperator inputOp = implementor.visitChild(this, 0, getChild());
     Project.Builder builder = Project.builder();
     builder.setInput(inputOp);
-    for (Pair<RexNode, String> pair : projects()) {
-      LogicalExpression expr = DrillOptiq.toDrill(implementor.getContext(), getChild(), pair.left);
-      builder.addExpr(new FieldReference("output." + pair.right), expr);
+    for (NamedExpression e: this.getProjectExpressions(implementor.getContext())) {
+      builder.addExpr(e);
     }
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
index bf5bcff..c9c8143 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRule.java
@@ -38,7 +38,7 @@ public class DrillProjectRule extends RelOptRule {
   public void onMatch(RelOptRuleCall call) {
     final ProjectRel project = (ProjectRel) call.rel(0);
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = project.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = project.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final RelNode convertedInput = convert(input, traits);
     call.transformTo(new DrillProjectRel(project.getCluster(), traits, convertedInput, project.getProjects(), project
         .getRowType()));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
index 63a7207..153b147 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillRel.java
@@ -27,7 +27,7 @@ import org.eigenbase.relopt.Convention;
 public interface DrillRel extends RelNode {
   /** Calling convention for relational expressions that are "implemented" by
    * generating Drill logical plans. */
-  Convention CONVENTION = new Convention.Impl("DRILL", DrillRel.class);
+  Convention DRILL_LOGICAL = new Convention.Impl("DRILL", DrillRel.class);
 
   LogicalOperator implement(DrillImplementor implementor);
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/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 b2757a3..5bd1386 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,6 +21,8 @@ import java.util.Iterator;
 
 import net.hydromatic.optiq.tools.RuleSet;
 
+import org.apache.drill.exec.planner.physical.ProjectPrule;
+import org.apache.drill.exec.planner.physical.ScanPrule;
 import org.eigenbase.rel.rules.MergeProjectRule;
 import org.eigenbase.rel.rules.PushFilterPastJoinRule;
 import org.eigenbase.rel.rules.PushFilterPastProjectRule;
@@ -44,25 +46,6 @@ public class DrillRuleSets {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class);
 
   public static final RuleSet DRILL_BASIC_RULES = new DrillRuleSet(ImmutableSet.of( //
-
-//      ExpandConversionRule.instance,
-//      SwapJoinRule.instance,
-//      RemoveDistinctRule.instance,
-//      UnionToDistinctRule.instance,
-//      RemoveTrivialProjectRule.instance,
-//      RemoveTrivialCalcRule.instance,
-//      RemoveSortRule.INSTANCE,
-//
-//      TableAccessRule.instance, //
-//      MergeProjectRule.instance, //
-//      PushFilterPastProjectRule.instance, //
-        PushFilterPastJoinRule.FILTER_ON_JOIN, //
-//      RemoveDistinctAggregateRule.instance, //
-//      ReduceAggregatesRule.instance, //
-//      SwapJoinRule.instance, //
-//      PushJoinThroughJoinRule.RIGHT, //
-//      PushJoinThroughJoinRule.LEFT, //
-//      PushSortPastProjectRule.INSTANCE, //
       
       DrillScanRule.INSTANCE,
       DrillFilterRule.INSTANCE,
@@ -75,6 +58,43 @@ public class DrillRuleSets {
       DrillUnionRule.INSTANCE
       ));
   
+  public static final RuleSet DRILL_PHYSICAL_MEM = new DrillRuleSet(ImmutableSet.of( //
+      DrillScanRule.INSTANCE,
+      DrillFilterRule.INSTANCE,
+      DrillProjectRule.INSTANCE,
+      DrillAggregateRule.INSTANCE,
+
+      DrillLimitRule.INSTANCE,
+      DrillSortRule.INSTANCE,
+      DrillJoinRule.INSTANCE,
+      DrillUnionRule.INSTANCE,
+      ProjectPrule.INSTANCE,
+      ScanPrule.INSTANCE
+
+//    ExpandConversionRule.instance,
+//    SwapJoinRule.instance,
+//    RemoveDistinctRule.instance,
+//    UnionToDistinctRule.instance,
+//    RemoveTrivialProjectRule.instance,
+//    RemoveTrivialCalcRule.instance,
+//    RemoveSortRule.INSTANCE,
+//
+//    TableAccessRule.instance, //
+//    MergeProjectRule.instance, //
+//    PushFilterPastProjectRule.instance, //
+//    PushFilterPastJoinRule.FILTER_ON_JOIN, //
+//    RemoveDistinctAggregateRule.instance, //
+//    ReduceAggregatesRule.instance, //
+//    SwapJoinRule.instance, //
+//    PushJoinThroughJoinRule.RIGHT, //
+//    PushJoinThroughJoinRule.LEFT, //
+//    PushSortPastProjectRule.INSTANCE, //      
+    ));
+  
+  public static final RuleSet DRILL_PHYSICAL_DISK = new DrillRuleSet(ImmutableSet.of( //
+      ProjectPrule.INSTANCE
+  
+    ));
   
   private static class DrillRuleSet implements RuleSet{
     final ImmutableSet<RelOptRule> rules;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
index afc2d1b..2629c77 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRel.java
@@ -18,11 +18,10 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Scan;
+import org.apache.drill.exec.planner.common.BaseScanRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
-import org.eigenbase.rel.TableAccessRelBase;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptTable;
 import org.eigenbase.relopt.RelTraitSet;
@@ -30,28 +29,20 @@ import org.eigenbase.relopt.RelTraitSet;
 /**
  * GroupScan of a Drill table.
  */
-public class DrillScanRel extends TableAccessRelBase implements DrillRel {
+public class DrillScanRel extends BaseScanRel implements DrillRel {
   private final DrillTable drillTable;
 
   /** Creates a DrillScan. */
   public DrillScanRel(RelOptCluster cluster, RelTraitSet traits, RelOptTable table) {
-    super(cluster, traits, table);
-    assert getConvention() == CONVENTION;
+    super(DRILL_LOGICAL, cluster, traits, table);
     this.drillTable = table.unwrap(DrillTable.class);
     assert drillTable != null;
   }
 
-//  @Override
-//  public void register(RelOptPlanner planner) {
-//    super.register(planner);
-//    DrillOptiq.registerStandardPlannerRules(planner);
-//  }
-
   public LogicalOperator implement(DrillImplementor implementor) {
     Scan.Builder builder = Scan.builder();
     builder.storageEngine(drillTable.getStorageEngineName());
     builder.selection(new JSONOptions(drillTable.getSelection()));
-    //builder.outputReference(new FieldReference("_MAP"));
     implementor.registerSource(drillTable);
     return builder.build();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
index 58e648a..e472daa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScanRule.java
@@ -19,6 +19,7 @@ package org.apache.drill.exec.planner.logical;
 
 import net.hydromatic.optiq.rules.java.JavaRules.EnumerableTableAccessRel;
 
+import org.apache.drill.exec.planner.common.BaseScanRel;
 import org.eigenbase.relopt.Convention;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.RelOptRuleCall;
@@ -31,13 +32,10 @@ public class DrillScanRule  extends RelOptRule {
     super(RelOptHelper.any(EnumerableTableAccessRel.class), "DrillTableRule");
   }
 
-
-
-
   @Override
   public void onMatch(RelOptRuleCall call) {
     final EnumerableTableAccessRel access = (EnumerableTableAccessRel) call.rel(0);
-    final RelTraitSet traits = access.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = access.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     call.transformTo(new DrillScanRel(access.getCluster(), traits, access.getTable()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
index 829947a..abfcd10 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillScreenRel.java
@@ -19,47 +19,20 @@ package org.apache.drill.exec.planner.logical;
 
 import java.util.List;
 
-import net.hydromatic.optiq.impl.java.JavaTypeFactory;
-import net.hydromatic.optiq.rules.java.EnumerableConvention;
-import net.hydromatic.optiq.rules.java.EnumerableRel;
-import net.hydromatic.optiq.rules.java.EnumerableRelImplementor;
-import net.hydromatic.optiq.rules.java.JavaRowFormat;
-import net.hydromatic.optiq.rules.java.PhysType;
-import net.hydromatic.optiq.rules.java.PhysTypeImpl;
-
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Store;
+import org.apache.drill.exec.planner.common.BaseScreenRel;
 import org.eigenbase.rel.RelNode;
-import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
-import org.eigenbase.relopt.RelOptCost;
-import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-/**
- * Relational expression that converts from Drill to Enumerable. At runtime it executes a Drill query and returns the
- * results as an {@link net.hydromatic.linq4j.Enumerable}.
- */
-public class DrillScreenRel extends SingleRel implements DrillRel {
+public class DrillScreenRel extends BaseScreenRel implements DrillRel {
   private static final Logger logger = LoggerFactory.getLogger(DrillScreenRel.class);
 
-  private PhysType physType;
-
   public DrillScreenRel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input) {
-    super(cluster, traitSet, input);
-    assert input.getConvention() == DrillRel.CONVENTION;
-    physType = PhysTypeImpl.of((JavaTypeFactory) cluster.getTypeFactory(), input.getRowType(), JavaRowFormat.ARRAY);
-  }
-
-  public PhysType getPhysType() {
-    return physType;
-  }
-
-  @Override
-  public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);
+    super(DRILL_LOGICAL, cluster, traitSet, input);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
index c968e85..9179668 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillSortRule.java
@@ -43,9 +43,9 @@ public class DrillSortRule extends RelOptRule {
     final SortRel sort = call.rel(0);
 
     final RelNode input = call.rel(1);
-    final RelTraitSet traits = sort.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = sort.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
 
-    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.CONVENTION));
+    final RelNode convertedInput = convert(input, input.getTraitSet().plus(DrillRel.DRILL_LOGICAL));
     call.transformTo(new DrillSortRel(sort.getCluster(), traits, convertedInput, sort.getCollation()));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
index 30c7810..8f9841c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillStoreRel.java
@@ -22,6 +22,8 @@ import java.util.List;
 import net.hydromatic.optiq.prepare.Prepare.CatalogReader;
 
 import org.apache.drill.common.logical.data.LogicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.TableModificationRelBase;
 import org.eigenbase.relopt.RelOptCluster;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
index bad6f74..c111306 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillTable.java
@@ -17,12 +17,18 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import java.io.IOException;
+
 import net.hydromatic.optiq.Schema.TableType;
 import net.hydromatic.optiq.Statistic;
 import net.hydromatic.optiq.Statistics;
 import net.hydromatic.optiq.Table;
 
+import org.apache.drill.common.JSONOptions;
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.store.StoragePlugin;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.RelOptTable;
 
@@ -32,12 +38,13 @@ public abstract class DrillTable implements Table{
   private final String storageEngineName;  
   public final StoragePluginConfig storageEngineConfig;
   private Object selection;
-  
+  private StoragePlugin plugin;
   
   /** Creates a DrillTable. */
-  public DrillTable(String storageEngineName, Object selection, StoragePluginConfig storageEngineConfig) {
+  public DrillTable(String storageEngineName, StoragePlugin plugin, Object selection) {
     this.selection = selection;
-    this.storageEngineConfig = storageEngineConfig;
+    
+    this.storageEngineConfig = plugin.getConfig();
     this.storageEngineName = storageEngineName;
   }
 
@@ -45,6 +52,10 @@ public abstract class DrillTable implements Table{
     return storageEngineConfig;
   }
   
+  public StoragePlugin getPlugin(){
+    return plugin;
+  }
+    
   public Object getSelection() {
     return selection;
   }
@@ -60,7 +71,7 @@ public abstract class DrillTable implements Table{
 
   public RelNode toRel(RelOptTable.ToRelContext context, RelOptTable table) {
     return new DrillScanRel(context.getCluster(),
-        context.getCluster().traitSetOf(DrillRel.CONVENTION),
+        context.getCluster().traitSetOf(DrillRel.DRILL_LOGICAL),
         table);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
index 1be9caf..c366341 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRel.java
@@ -24,6 +24,8 @@ import net.hydromatic.linq4j.Ord;
 import org.apache.drill.common.logical.data.Limit;
 import org.apache.drill.common.logical.data.LogicalOperator;
 import org.apache.drill.common.logical.data.Union;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
index bc1e6f4..99f2c98 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillUnionRule.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.logical;
 
+import org.apache.drill.exec.planner.common.DrillUnionRel;
 import org.eigenbase.rel.UnionRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.*;
@@ -37,7 +38,7 @@ public class DrillUnionRule extends RelOptRule {
   @Override
   public void onMatch(RelOptRuleCall call) {
     final UnionRel union = (UnionRel) call.rel(0);
-    final RelTraitSet traits = union.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = union.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     final List<RelNode> convertedInputs = new ArrayList<>();
     for (RelNode input : union.getInputs()) {
       final RelNode convertedInput = convert(input, traits);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
index e770181..a3551e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRel.java
@@ -35,7 +35,7 @@ import org.eigenbase.rex.RexLiteral;
 public class DrillValuesRel extends ValuesRelBase implements DrillRel {
   protected DrillValuesRel(RelOptCluster cluster, RelDataType rowType, List<List<RexLiteral>> tuples, RelTraitSet traits) {
     super(cluster, rowType, tuples, traits);
-    assert getConvention() == CONVENTION;
+    assert getConvention() == DRILL_LOGICAL;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
index fae18ca..8a8cb1f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillValuesRule.java
@@ -33,7 +33,7 @@ public class DrillValuesRule extends RelOptRule {
   @Override
   public void onMatch(RelOptRuleCall call) {
     final ValuesRel values = (ValuesRel) call.rel(0);
-    final RelTraitSet traits = values.getTraitSet().plus(DrillRel.CONVENTION);
+    final RelTraitSet traits = values.getTraitSet().plus(DrillRel.DRILL_LOGICAL);
     call.transformTo(new DrillValuesRel(values.getCluster(), values.getRowType(), values.getTuples(), traits));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
index 36b554f..d309405 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DynamicDrillTable.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.logical;
 
 import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.store.dfs.FileSystemPlugin;
 import org.eigenbase.reltype.RelDataType;
 import org.eigenbase.reltype.RelDataTypeFactory;
 
@@ -26,8 +27,8 @@ public class DynamicDrillTable extends DrillTable{
 
   private RelDataTypeHolder holder = new RelDataTypeHolder();
   
-  public DynamicDrillTable(String storageEngineName, Object selection, StoragePluginConfig storageEngineConfig) {
-    super(storageEngineName, selection, storageEngineConfig);
+  public DynamicDrillTable(FileSystemPlugin plugin, String storageEngineName, Object selection, StoragePluginConfig storageEngineConfig) {
+    super(storageEngineName, plugin, selection);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
index 2fcf660..d54ee79 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/EnumerableDrillRule.java
@@ -19,19 +19,20 @@ package org.apache.drill.exec.planner.logical;
 
 import net.hydromatic.optiq.rules.java.EnumerableConvention;
 
+import org.apache.drill.exec.planner.common.BaseScreenRel;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.convert.ConverterRule;
 
 /**
- * Rule that converts any Drill relational expression to enumerable format by adding a {@link DrillScreenRel}.
+ * Rule that converts any Drill relational expression to enumerable format by adding a {@link BaseScreenRel}.
  */
 public class EnumerableDrillRule extends ConverterRule {
 
-  public static EnumerableDrillRule INSTANCE = new EnumerableDrillRule(EnumerableConvention.INSTANCE);
+  public static EnumerableDrillRule INSTANCE = new EnumerableDrillRule();
   
   
-  private EnumerableDrillRule(EnumerableConvention outConvention) {
-    super(RelNode.class, DrillRel.CONVENTION, outConvention, "EnumerableDrillRule." + outConvention);
+  private EnumerableDrillRule() {
+    super(RelNode.class, DrillRel.DRILL_LOGICAL, EnumerableConvention.INSTANCE, "EnumerableDrillRule.");
   }
 
   @Override
@@ -41,7 +42,7 @@ public class EnumerableDrillRule extends ConverterRule {
 
   @Override
   public RelNode convert(RelNode rel) {
-    assert rel.getTraitSet().contains(DrillRel.CONVENTION);
+    assert rel.getTraitSet().contains(DrillRel.DRILL_LOGICAL);
     return new DrillScreenRel(rel.getCluster(), rel.getTraitSet().replace(getOutConvention()), rel);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
new file mode 100644
index 0000000..27b6be9
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashPrel.java
@@ -0,0 +1,37 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class HashPrel extends AggregateRelBase implements Prel{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashPrel.class);
+
+  public HashPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
+      List<AggregateCall> aggCalls) throws InvalidRelException {
+    super(cluster, traits, child, groupSet, aggCalls);
+    for (AggregateCall aggCall : aggCalls) {
+      if (aggCall.isDistinct()) {
+        throw new InvalidRelException("HashPrel does not support DISTINCT aggregates");
+      }
+    }
+    assert getConvention() == DRILL_PHYSICAL;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    try {
+      return new HashPrel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+    } catch (InvalidRelException e) {
+      throw new AssertionError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
new file mode 100644
index 0000000..585f9a0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -0,0 +1,20 @@
+package org.apache.drill.exec.planner.physical;
+
+import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rex.RexNode;
+
+public class PhysicalPlanCreator {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PhysicalPlanCreator.class);
+  
+  public LogicalExpression getExpression(RexNode e, RelNode input){
+    return null;
+  }
+  
+  public DrillParseContext getContext(){
+    return null;
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
new file mode 100644
index 0000000..6640aef
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/Prel.java
@@ -0,0 +1,17 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+
+public interface Prel extends RelNode {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Prel.class);
+  
+  final static Convention DRILL_PHYSICAL = new Convention.Impl("DRILL_PHYSICAL", Prel.class);
+  
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException;
+    
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
new file mode 100644
index 0000000..b7e18e8
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -0,0 +1,38 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.Project;
+import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.rex.RexNode;
+
+public class ProjectPrel extends BaseProjectRel implements Prel{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPrel.class);
+  
+  
+  protected ProjectPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
+      RelDataType rowType) {
+    super(DRILL_PHYSICAL, cluster, traits, child, exps, rowType);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new ProjectPrel(getCluster(), traitSet, sole(inputs), new ArrayList<RexNode>(exps), rowType);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    Project p = new Project(this.getProjectExpressions(creator.getContext()), child.getPhysicalOperator(creator));
+    return p;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
new file mode 100644
index 0000000..eb1efce
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrule.java
@@ -0,0 +1,45 @@
+/**
+ * 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.physical;
+
+import org.apache.drill.exec.planner.common.BaseProjectRel;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.rel.ProjectRel;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.Convention;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class ProjectPrule extends RelOptRule {
+  public static final RelOptRule INSTANCE = new ProjectPrule();
+
+  private ProjectPrule() {
+    super(RelOptHelper.some(BaseProjectRel.class, DrillRel.DRILL_LOGICAL, RelOptHelper.any(RelNode.class)), "ProjectPrule");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final BaseProjectRel project = (BaseProjectRel) call.rel(0);
+    final RelNode input = call.rel(1);
+    final RelTraitSet traits = project.getTraitSet().replace(Prel.DRILL_PHYSICAL);
+    final RelNode convertedInput = convert(input, traits);
+    call.transformTo(new ProjectPrel(project.getCluster(), traits, convertedInput, project.getProjects(), project.getRowType()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
new file mode 100644
index 0000000..b4906f1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -0,0 +1,29 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.io.IOException;
+
+import org.apache.drill.common.JSONOptions;
+import org.apache.drill.exec.physical.base.GroupScan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.store.StoragePlugin;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptTable;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class ScanPrel extends BaseScanRel implements Prel{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScanPrel.class);
+
+  public ScanPrel(RelOptCluster cluster, RelTraitSet traits, RelOptTable tbl) {
+    super(DRILL_PHYSICAL, cluster, traits, tbl);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    StoragePlugin plugin = this.drillTable.getPlugin();
+    GroupScan scan = plugin.getPhysicalScan(new JSONOptions(drillTable.getSelection()));
+    return scan;    
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.java
new file mode 100644
index 0000000..568f8e0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrule.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.planner.physical;
+
+import org.apache.drill.exec.planner.common.BaseScanRel;
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class ScanPrule extends RelOptRule{
+  public static final RelOptRule INSTANCE = new ScanPrule();
+
+  
+  public ScanPrule() {
+    super(RelOptHelper.any(BaseScanRel.class), "Prel.ScanRule");
+    
+  }
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final BaseScanRel scan = (BaseScanRel) call.rel(0);
+    final RelTraitSet traits = scan.getTraitSet().replace(Prel.DRILL_PHYSICAL);
+    call.transformTo(new ScanPrel(scan.getCluster(), traits, scan.getTable()));
+  }
+
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
new file mode 100644
index 0000000..4868c66
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
@@ -0,0 +1,25 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.util.List;
+
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class ScreenPrel extends SingleRel implements Prel{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenPrel.class);
+
+  
+  public ScreenPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
+    super(cluster, traits, child);
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new ScreenPrel(getCluster(), traitSet, sole(inputs));
+  }
+  
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
new file mode 100644
index 0000000..82c7b33
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -0,0 +1,36 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class SortPrel extends AggregateRelBase{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortPrel.class);
+
+  public SortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
+      List<AggregateCall> aggCalls) throws InvalidRelException {
+    super(cluster, traits, child, groupSet, aggCalls);
+    for (AggregateCall aggCall : aggCalls) {
+      if (aggCall.isDistinct()) {
+        throw new InvalidRelException("DrillAggregateRel does not support DISTINCT aggregates");
+      }
+    }
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    try {
+      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+    } catch (InvalidRelException e) {
+      throw new AssertionError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
new file mode 100644
index 0000000..820ff48
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StorageRule.java
@@ -0,0 +1,5 @@
+package org.apache.drill.exec.planner.physical;
+
+public class StorageRule {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StorageRule.class);
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
new file mode 100644
index 0000000..01b6164
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -0,0 +1,36 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class StreamAggPrel extends AggregateRelBase{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(StreamAggPrel.class);
+
+  public StreamAggPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
+      List<AggregateCall> aggCalls) throws InvalidRelException {
+    super(cluster, traits, child, groupSet, aggCalls);
+    for (AggregateCall aggCall : aggCalls) {
+      if (aggCall.isDistinct()) {
+        throw new InvalidRelException("DrillAggregateRel does not support DISTINCT aggregates");
+      }
+    }
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    try {
+      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+    } catch (InvalidRelException e) {
+      throw new AssertionError(e);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/22c41907/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
new file mode 100644
index 0000000..027196e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNSortPrel.java
@@ -0,0 +1,36 @@
+package org.apache.drill.exec.planner.physical;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.apache.drill.exec.planner.logical.DrillAggregateRel;
+import org.eigenbase.rel.AggregateCall;
+import org.eigenbase.rel.AggregateRelBase;
+import org.eigenbase.rel.InvalidRelException;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelTraitSet;
+
+public class TopNSortPrel extends AggregateRelBase{
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TopNSortPrel.class);
+
+  public TopNSortPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, BitSet groupSet,
+      List<AggregateCall> aggCalls) throws InvalidRelException {
+    super(cluster, traits, child, groupSet, aggCalls);
+    for (AggregateCall aggCall : aggCalls) {
+      if (aggCall.isDistinct()) {
+        throw new InvalidRelException("DrillAggregateRel does not support DISTINCT aggregates");
+      }
+    }
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    try {
+      return new DrillAggregateRel(getCluster(), traitSet, sole(inputs), getGroupSet(), aggCalls);
+    } catch (InvalidRelException e) {
+      throw new AssertionError(e);
+    }
+  }
+}


[44/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
index 82f21eb..4935e58 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -39,7 +39,7 @@ import org.eigenbase.relopt.RelTraitSet;
 public class HashToRandomExchangePrel extends SingleRel implements Prel {
 
   private final List<DistributionField> fields;
-  
+
   public HashToRandomExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, List<DistributionField> fields) {
     super(cluster, traitSet, input);
     this.fields = fields;
@@ -48,7 +48,7 @@ public class HashToRandomExchangePrel extends SingleRel implements Prel {
 
   @Override
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    return super.computeSelfCost(planner).multiplyBy(.1);    
+    return super.computeSelfCost(planner).multiplyBy(.1);
     //return planner.getCostFactory().makeZeroCost();
   }
 
@@ -56,27 +56,25 @@ public class HashToRandomExchangePrel extends SingleRel implements Prel {
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new HashToRandomExchangePrel(getCluster(), traitSet, sole(inputs), fields);
   }
-  
+
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
+    if(PlanningSettings.get(getCluster()).isSingleMode()) return childPOP;
+
     //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
-      childPOP = new SelectionVectorRemover(childPOP);
-      creator.addPhysicalOperator(childPOP);
-    }
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE);
 
     HashToRandomExchange g = new HashToRandomExchange(childPOP, PrelUtil.getHashExpression(this.fields, getChild().getRowType()));
-    creator.addPhysicalOperator(g);
-    return g;    
+    return g;
   }
-  
+
   public List<DistributionField> getFields() {
     return this.fields;
   }
-  
+
   @Override
   public RelWriter explainTerms(RelWriter pw) {
     super.explainTerms(pw);
@@ -84,6 +82,6 @@ public class HashToRandomExchangePrel extends SingleRel implements Prel {
         pw.item("dist" + ord.i, ord.e);
       }
     return pw;
-  }  
-  
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
index f2f46c1..7c8d767 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/LimitPrel.java
@@ -52,9 +52,9 @@ public class LimitPrel extends DrillLimitRelBase implements Prel {
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
     // First offset to include into results (inclusive). Null implies it is starting from offset 0
     int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
 
@@ -63,16 +63,14 @@ public class LimitPrel extends DrillLimitRelBase implements Prel {
     Integer last = fetch != null ? Math.max(0, RexLiteral.intValue(fetch)) + first : null;
 
     Limit limit = new Limit(childPOP, first, last);
-    
-    creator.addPhysicalOperator(limit);
-    
+
     return limit;
   }
-  
+
 //  @Override
 //  public LogicalOperator implement(DrillImplementor implementor) {
 //    LogicalOperator inputOp = implementor.visitChild(this, 0, getChild());
-//    
+//
 //    // First offset to include into results (inclusive). Null implies it is starting from offset 0
 //    int first = offset != null ? Math.max(0, RexLiteral.intValue(offset)) : 0;
 //
@@ -83,7 +81,7 @@ public class LimitPrel extends DrillLimitRelBase implements Prel {
 //    limit.setInput(inputOp);
 //    return limit;
 //  }
-  
+
 //  public static LimitPrel convert(Limit limit, ConversionContext context) throws InvalidRelException{
 //    RelNode input = context.toRel(limit.getInput());
 //    RexNode first = context.getRexBuilder().makeExactLiteral(BigDecimal.valueOf(limit.getFirst()), context.getTypeFactory().createSqlType(SqlTypeName.INTEGER));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
index 0ced205..978a531 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/MergeJoinPrel.java
@@ -44,7 +44,7 @@ import com.beust.jcommander.internal.Lists;
 public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
 
   //private final JoinCondition[] joinConditions; // Drill's representation of join conditions
-  
+
   /** Creates a MergeJoiPrel. */
   public MergeJoinPrel(RelOptCluster cluster, RelTraitSet traits, RelNode left, RelNode right, RexNode condition,
       JoinRelType joinType) throws InvalidRelException {
@@ -53,15 +53,15 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     if (condition.isAlwaysTrue()) {
       throw new InvalidRelException("MergeJoinPrel does not support cartesian product join");
     }
-    
+
     RexNode remaining = RelOptUtil.splitJoinCondition(left, right, condition, leftKeys, rightKeys);
     if (!remaining.isAlwaysTrue() && (leftKeys.size() == 0 || rightKeys.size() == 0)) {
       throw new InvalidRelException("MergeJoinPrel only supports equi-join");
     }
   }
 
-  
-  @Override 
+
+  @Override
   public JoinRelBase copy(RelTraitSet traitSet, RexNode conditionExpr, RelNode left, RelNode right, JoinRelType joinType) {
     try {
       return new MergeJoinPrel(this.getCluster(), traitSet, left, right, conditionExpr, joinType);
@@ -70,8 +70,8 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
     }
   }
 
-  @Override  
-  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {    
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     final List<String> fields = getRowType().getFieldNames();
     assert isUnique(fields);
     final int leftCount = left.getRowType().getFieldCount();
@@ -80,41 +80,34 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
 
     PhysicalOperator leftPop = implementInput(creator, 0, left);
     PhysicalOperator rightPop = implementInput(creator, leftCount, right);
-   
+
     //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
-    if (leftPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
-      leftPop = new SelectionVectorRemover(leftPop);
-      creator.addPhysicalOperator(leftPop);
-    }
+    leftPop = PrelUtil.removeSvIfRequired(leftPop, SelectionVectorMode.NONE, SelectionVectorMode.TWO_BYTE);
 
     //Currently, only accepts "NONE" or "SV2". For other, requires SelectionVectorRemover
-    if (rightPop.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
-      rightPop = new SelectionVectorRemover(rightPop);
-      creator.addPhysicalOperator(rightPop);
-    }
-    
+    rightPop = PrelUtil.removeSvIfRequired(rightPop, SelectionVectorMode.NONE, SelectionVectorMode.TWO_BYTE);
+
     JoinRelType jtype = this.getJoinType();
-            
+
     List<JoinCondition> conditions = Lists.newArrayList();
-    
+
     for (Pair<Integer, Integer> pair : Pair.zip(leftKeys, rightKeys)) {
       conditions.add(new JoinCondition("==", new FieldReference(leftFields.get(pair.left)), new FieldReference(rightFields.get(pair.right))));
     }
-    
+
     MergeJoinPOP mjoin = new MergeJoinPOP(leftPop, rightPop, conditions, jtype);
-    creator.addPhysicalOperator(mjoin);
-   
+
     return mjoin;
   }
 
   public List<Integer> getLeftKeys() {
     return this.leftKeys;
   }
-  
+
   public List<Integer> getRightKeys() {
     return this.rightKeys;
   }
-  
+
   /**
    * Check to make sure that the fields of the inputs are the same as the output field names.  If not, insert a project renaming them.
    * @param implementor
@@ -124,7 +117,7 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
    * @return
    */
   private PhysicalOperator implementInput(PhysicalPlanCreator creator, int offset, RelNode input) throws IOException {
-    final PhysicalOperator inputOp = ((Prel) input).getPhysicalOperator(creator); 
+    final PhysicalOperator inputOp = ((Prel) input).getPhysicalOperator(creator);
     assert uniqueFieldNames(input.getRowType());
     final List<String> fields = getRowType().getFieldNames();
     final List<String> inputFields = input.getRowType().getFieldNames();
@@ -141,22 +134,18 @@ public class MergeJoinPrel  extends DrillJoinRelBase implements Prel {
 
   private PhysicalOperator rename(PhysicalPlanCreator creator, PhysicalOperator inputOp, List<String> inputFields, List<String> outputFields) {
     List<NamedExpression> exprs = Lists.newArrayList();
-    
+
     //Currently, Project only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!inputOp.getSVMode().equals(SelectionVectorMode.NONE)) {
-      inputOp = new SelectionVectorRemover(inputOp);
-      creator.addPhysicalOperator(inputOp);
-    }
+    inputOp = PrelUtil.removeSvIfRequired(inputOp, SelectionVectorMode.NONE);
 
     for (Pair<String, String> pair : Pair.zip(inputFields, outputFields)) {
-      exprs.add(new NamedExpression(new FieldReference(pair.left), new FieldReference("output." + pair.right)));
+      exprs.add(new NamedExpression(new FieldReference(pair.left), new FieldReference(pair.right)));
     }
-    
+
     Project proj = new Project(exprs, inputOp);
-    
-    creator.addPhysicalOperator(proj);
+
     return proj;
   }
 
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
index 0c9bf34..9ac07d3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PhysicalPlanCreator.java
@@ -38,47 +38,43 @@ public class PhysicalPlanCreator {
   private List<PhysicalOperator> popList;
   private final QueryContext context;
   PhysicalPlan plan = null;
-  
+
   public PhysicalPlanCreator(QueryContext context) {
     this.context = context;
     popList = Lists.newArrayList();
   }
-  
+
   public QueryContext getContext() {
     return context;
   }
-  
-  public void addPhysicalOperator(PhysicalOperator op) {
-    popList.add(op);  
-  }
-  
+
   public PhysicalPlan build(Prel rootPrel, boolean forceRebuild) {
 
     if (plan != null && !forceRebuild) {
       return plan;
     }
-    
+
     PlanPropertiesBuilder propsBuilder = PlanProperties.builder();
     propsBuilder.type(PlanType.APACHE_DRILL_PHYSICAL);
     propsBuilder.version(1);
     propsBuilder.resultMode(ResultMode.EXEC);
     propsBuilder.generator(PhysicalPlanCreator.class.getName(), "");
 
-    
-    try { 
-      // invoke getPhysicalOperator on the root Prel which will recursively invoke it 
+
+    try {
+      // invoke getPhysicalOperator on the root Prel which will recursively invoke it
       // on the descendants and we should have a well-formed physical operator tree
       PhysicalOperator rootPOP = rootPrel.getPhysicalOperator(this);
       if (rootPOP != null) {
-        assert (popList.size() > 0); //getPhysicalOperator() is supposed to populate this list 
+        assert (popList.size() > 0); //getPhysicalOperator() is supposed to populate this list
         plan = new PhysicalPlan(propsBuilder.build(), popList);
       }
-      
+
     } catch (IOException e) {
       plan = null;
       throw new UnsupportedOperationException("Physical plan created failed with error : " + e.toString());
     }
-    
+
     return plan;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlanningSettings.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlanningSettings.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlanningSettings.java
new file mode 100644
index 0000000..ae293a2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PlanningSettings.java
@@ -0,0 +1,57 @@
+/**
+ * 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.physical;
+
+import org.eigenbase.relopt.RelOptCluster;
+
+public class PlanningSettings {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PlanningSettings.class);
+
+  private static ThreadLocal<PlanningSettings> settings = new ThreadLocal<>();
+
+  private boolean singleMode;
+
+  public boolean isSingleMode() {
+    return singleMode;
+  }
+
+  public void setSingleMode(boolean singleMode) {
+    this.singleMode = singleMode;
+  }
+
+  /**
+   * Convenience method to extract planning settings from RelOptCluster. Uses threadlocal until Optiq supports
+   * passthrough.
+   */
+  public static PlanningSettings get(RelOptCluster cluster) {
+    PlanningSettings s = settings.get();
+    if (s == null) {
+      s = new PlanningSettings();
+      settings.set(s);
+    }
+
+    return s;
+  }
+
+
+  public static PlanningSettings get(){
+    return get(null);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index 45e7523..5dcf9aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -24,8 +24,10 @@ import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.FunctionCall;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.logical.data.Order.Ordering;
-import org.apache.drill.exec.expr.fn.impl.HashFunctions;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
+import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelFieldCollation;
 import org.eigenbase.reltype.RelDataType;
@@ -37,14 +39,14 @@ public class PrelUtil {
 
   public static List<Ordering> getOrdering(RelCollation collation, RelDataType rowType) {
     List<Ordering> orderExpr = Lists.newArrayList();
-    
+
     final List<String> childFields = rowType.getFieldNames();
-    
-    for (RelFieldCollation fc: collation.getFieldCollations() ) {      
+
+    for (RelFieldCollation fc: collation.getFieldCollations() ) {
       FieldReference fr = new FieldReference(childFields.get(fc.getFieldIndex()), ExpressionPosition.UNKNOWN);
       orderExpr.add(new Ordering(fc.getDirection(), fr, fc.nullDirection));
     }
-    
+
     return orderExpr;
   }
 
@@ -53,20 +55,31 @@ public class PrelUtil {
    */
   public static LogicalExpression getHashExpression(List<DistributionField> fields, RelDataType rowType) {
     assert fields.size() > 0;
-    
+
     final List<String> childFields = rowType.getFieldNames();
-    
-    FieldReference fr = new FieldReference(childFields.get(fields.get(0).getFieldId()), ExpressionPosition.UNKNOWN);    
+
+    FieldReference fr = new FieldReference(childFields.get(fields.get(0).getFieldId()), ExpressionPosition.UNKNOWN);
     FunctionCall func = new FunctionCall("hash",  ImmutableList.of((LogicalExpression)fr), ExpressionPosition.UNKNOWN);
-    
-    for (int i = 1; i<fields.size(); i++) {     
-      fr = new FieldReference(childFields.get(fields.get(i).getFieldId()), ExpressionPosition.UNKNOWN);      
+
+    for (int i = 1; i<fields.size(); i++) {
+      fr = new FieldReference(childFields.get(fields.get(i).getFieldId()), ExpressionPosition.UNKNOWN);
       FunctionCall func2 = new FunctionCall("hash",  ImmutableList.of((LogicalExpression)fr), ExpressionPosition.UNKNOWN);
-      
+
       func = new FunctionCall("xor", ImmutableList.of((LogicalExpression)func, (LogicalExpression)func2), ExpressionPosition.UNKNOWN);
     }
-    
+
     return func;
   }
 
+
+  public static PhysicalOperator removeSvIfRequired(PhysicalOperator child, SelectionVectorMode... allowed){
+    SelectionVectorMode current = child.getSVMode();
+    for(SelectionVectorMode m : allowed){
+      if(current == m) return child;
+    }
+    return new SelectionVectorRemover(child);
+  }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
index 7a9ad43..4fb29fd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ProjectPrel.java
@@ -36,8 +36,8 @@ import org.eigenbase.rex.RexNode;
 
 public class ProjectPrel extends DrillProjectRelBase implements Prel{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ProjectPrel.class);
-  
-  
+
+
   protected ProjectPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
       RelDataType rowType) {
     super(DRILL_PHYSICAL, cluster, traits, child, exps, rowType);
@@ -51,18 +51,14 @@ public class ProjectPrel extends DrillProjectRelBase implements Prel{
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
     //Currently, Project only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
-      childPOP = new SelectionVectorRemover(childPOP);
-      creator.addPhysicalOperator(childPOP);
-    }
-    
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE);
+
     Project p = new Project(this.getProjectExpressions(new DrillParseContext()),  childPOP);
-    creator.addPhysicalOperator(p);
-    
+
     return p;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
index c411649..a945129 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScanPrel.java
@@ -38,7 +38,7 @@ public class ScanPrel extends DrillScanRelBase implements Prel{
     super(DRILL_PHYSICAL, cluster, traits, tbl);
   }
 
-  
+
   @Override
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return super.copy(traitSet, inputs);
@@ -55,10 +55,9 @@ public class ScanPrel extends DrillScanRelBase implements Prel{
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     StoragePlugin plugin = this.drillTable.getPlugin();
     GroupScan scan = plugin.getPhysicalScan(new JSONOptions(drillTable.getSelection()));
-    creator.addPhysicalOperator(scan);
-    
-    return scan;    
+
+    return scan;
   }
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
index 6819f31..c3b452f 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/ScreenPrel.java
@@ -33,7 +33,7 @@ public class ScreenPrel extends DrillScreenRelBase implements Prel {
 
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ScreenPrel.class);
 
-  
+
   public ScreenPrel(RelOptCluster cluster, RelTraitSet traits, RelNode child) {
     super(Prel.DRILL_PHYSICAL, cluster, traits, child);
   }
@@ -42,22 +42,18 @@ public class ScreenPrel extends DrillScreenRelBase implements Prel {
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new ScreenPrel(getCluster(), traitSet, sole(inputs));
   }
-  
-  @Override  
+
+  @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
     //Currently, Screen only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
-      childPOP = new SelectionVectorRemover(childPOP);
-      creator.addPhysicalOperator(childPOP);
-    }
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE);
 
     Screen s = new Screen(childPOP, creator.getContext().getCurrentEndpoint());
-    creator.addPhysicalOperator(s);
-    return s; 
+    return s;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
index 0d41a71..f5c2547 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
@@ -44,9 +44,9 @@ import org.eigenbase.relopt.RelTraitSet;
 import com.beust.jcommander.internal.Lists;
 
 public class SingleMergeExchangePrel extends SingleRel implements Prel {
-  
-  private final RelCollation collation ; 
-  
+
+  private final RelCollation collation ;
+
   public SingleMergeExchangePrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode input, RelCollation collation) {
     super(cluster, traitSet, input);
     this.collation = collation;
@@ -63,23 +63,21 @@ public class SingleMergeExchangePrel extends SingleRel implements Prel {
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new SingleMergeExchangePrel(getCluster(), traitSet, sole(inputs), collation);
   }
-  
-  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {    
+
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
+    if(PlanningSettings.get(getCluster()).isSingleMode()) return childPOP;
+
     //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
-      childPOP = new SelectionVectorRemover(childPOP);
-      creator.addPhysicalOperator(childPOP);
-    }
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE);
 
     SingleMergeExchange g = new SingleMergeExchange(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()));
-    creator.addPhysicalOperator(g);
-    return g;    
+    return g;
   }
-  
+
   @Override
   public RelWriter explainTerms(RelWriter pw) {
     super.explainTerms(pw);
@@ -91,6 +89,6 @@ public class SingleMergeExchangePrel extends SingleRel implements Prel {
       }
     }
     return pw;
-  }  
-  
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index 9c73d77..344be4e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -48,18 +48,16 @@ public class SortPrel extends SortRel implements Prel {
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
-    if (childPOP.getSVMode().equals(SelectionVectorMode.FOUR_BYTE)) {
-      throw new UnsupportedOperationException();
-    }
-    
-    Sort g = new ExternalSort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
-    
-    creator.addPhysicalOperator(g);
-    
-    return g;    
+
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE);
+    Sort g = new Sort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
+
+//    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE, SelectionVectorMode.TWO_BYTE);
+//    Sort g = new ExternalSort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
+
+    return g;
   }
 
   public SortPrel copy(
@@ -70,11 +68,11 @@ public class SortPrel extends SortRel implements Prel {
       RexNode fetch) {
     return new SortPrel(getCluster(), traitSet, newInput, newCollation);
   }
-  
+
   @Override
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice. 
+    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice.
     return super.computeSelfCost(planner).multiplyBy(0.1);
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
index b9f8c41..c2880da 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/StreamAggPrel.java
@@ -67,21 +67,21 @@ public class StreamAggPrel extends AggregateRelBase implements Prel{
       throw new AssertionError(e);
     }
   }
-   
+
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     // Prel child = (Prel) this.getChild();
-    
+
     final List<String> childFields = getChild().getRowType().getFieldNames();
     final List<String> fields = getRowType().getFieldNames();
     List<NamedExpression> keys = Lists.newArrayList();
     List<NamedExpression> exprs = Lists.newArrayList();
-    
+
     for (int group : BitSets.toIter(groupSet)) {
       FieldReference fr = new FieldReference(childFields.get(group), ExpressionPosition.UNKNOWN);
       keys.add(new NamedExpression(fr, fr));
     }
-    
+
     for (Ord<AggregateCall> aggCall : Ord.zip(aggCalls)) {
       FieldReference ref = new FieldReference(fields.get(groupSet.cardinality() + aggCall.i));
       LogicalExpression expr = toDrill(aggCall.e, childFields, new DrillParseContext());
@@ -90,22 +90,21 @@ public class StreamAggPrel extends AggregateRelBase implements Prel{
 
     Prel child = (Prel) this.getChild();
     StreamingAggregate g = new StreamingAggregate(child.getPhysicalOperator(creator), keys.toArray(new NamedExpression[keys.size()]), exprs.toArray(new NamedExpression[exprs.size()]), 1.0f);
-    creator.addPhysicalOperator(g);
-    
-    return g;    
+
+    return g;
 
   }
-  
+
   private LogicalExpression toDrill(AggregateCall call, List<String> fn, DrillParseContext pContext) {
     List<LogicalExpression> args = Lists.newArrayList();
     for(Integer i : call.getArgList()){
       args.add(new FieldReference(fn.get(i)));
     }
-    
+
     // for count(1).
     if(args.isEmpty()) args.add(new ValueExpressions.LongExpression(1l));
     LogicalExpression expr = new FunctionCall(call.getAggregation().getName().toLowerCase(), args, ExpressionPosition.UNKNOWN );
     return expr;
   }
- 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
index 7c811f7..e981a45 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
@@ -37,7 +37,7 @@ public class TopNPrel extends SingleRel implements Prel {
 
   protected int limit;
   protected final RelCollation collation;
-  
+
   public TopNPrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, int limit, RelCollation collation) {
     super(cluster, traitSet, child);
     this.limit = limit;
@@ -52,23 +52,21 @@ public class TopNPrel extends SingleRel implements Prel {
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-     
+
     TopN topN = new TopN(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false, this.limit);
-    
-    creator.addPhysicalOperator(topN);
-    
+
     return topN;
   }
-  
+
   @Override
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
-    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice. 
+    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice.
     return super.computeSelfCost(planner).multiplyBy(0.05);
   }
 
-  
+
   @Override
   public RelWriter explainTerms(RelWriter pw) {
     return super.explainTerms(pw)

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
index 38812c7..a1951f6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/UnionExchangePrel.java
@@ -49,21 +49,19 @@ public class UnionExchangePrel extends SingleRel implements Prel {
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new UnionExchangePrel(getCluster(), traitSet, sole(inputs));
   }
-  
+
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-     
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
+
+    if(PlanningSettings.get(getCluster()).isSingleMode()) return childPOP;
+
     //Currently, only accepts "NONE". For other, requires SelectionVectorRemover
-    if (!childPOP.getSVMode().equals(SelectionVectorMode.NONE)) {
-      childPOP = new SelectionVectorRemover(childPOP);
-      creator.addPhysicalOperator(childPOP);
-    }
-   
+    childPOP = PrelUtil.removeSvIfRequired(childPOP, SelectionVectorMode.NONE, SelectionVectorMode.TWO_BYTE);
+
     UnionExchange g = new UnionExchange(childPOP);
-    creator.addPhysicalOperator(g);
-    return g;    
+    return g;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
new file mode 100644
index 0000000..d40e0d7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DirectPlan.java
@@ -0,0 +1,63 @@
+/**
+ * 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;
+
+import java.util.Collections;
+import java.util.Iterator;
+
+import org.apache.drill.common.logical.PlanProperties;
+import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.common.logical.PlanProperties.PlanPropertiesBuilder;
+import org.apache.drill.common.logical.PlanProperties.PlanType;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.config.Screen;
+import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.SimpleCommandResult;
+import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
+import org.apache.drill.exec.store.direct.DirectGroupScan;
+import org.apache.drill.exec.store.pojo.PojoRecordReader;
+
+public class DirectPlan {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DirectPlan.class);
+
+
+  public static PhysicalPlan createDirectPlan(QueryContext context, boolean result, String message){
+    return createDirectPlan(context, new SimpleCommandResult(result, message));
+  }
+
+  @SuppressWarnings("unchecked")
+  public static <T> PhysicalPlan createDirectPlan(QueryContext context, T obj){
+    Iterator<T> iter = (Iterator<T>) Collections.singleton(obj).iterator();
+    return createDirectPlan(context.getCurrentEndpoint(), iter, (Class<T>) obj.getClass());
+
+  }
+  public static <T> PhysicalPlan createDirectPlan(DrillbitEndpoint endpoint, Iterator<T> iterator, Class<T> clazz){
+    PojoRecordReader<T> reader = new PojoRecordReader<T>(clazz, iterator);
+    DirectGroupScan scan = new DirectGroupScan(reader);
+    Screen screen = new Screen(scan, endpoint);
+
+    PlanPropertiesBuilder propsBuilder = PlanProperties.builder();
+    propsBuilder.type(PlanType.APACHE_DRILL_PHYSICAL);
+    propsBuilder.version(1);
+    propsBuilder.resultMode(ResultMode.EXEC);
+    propsBuilder.generator(DirectPlan.class.getSimpleName(), "");
+    return new PhysicalPlan(propsBuilder.build(), DefaultSqlHandler.getPops(screen));
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
index 0dbbc56..7c8bce2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
@@ -66,26 +66,26 @@ public class DrillOperatorTable extends SqlStdOperatorTable {
     // TODO: add hive functions.
   }
 
+
+
   @Override
-  public List<SqlOperator> lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax) {
+  public void lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax, List<SqlOperator> operatorList) {
     if (syntax == SqlSyntax.FUNCTION) {
-      List<SqlOperator> drillOps = opMap.get(opName.getSimple());
-      if (drillOps == null || drillOps.isEmpty())
-        return inner.lookupOperatorOverloads(opName, category, syntax);
 
-      List<SqlOperator> optiqOps = inner.lookupOperatorOverloads(opName, category, syntax);
-      if (optiqOps.isEmpty())
-        return drillOps;
+      // add optiq.
+      inner.lookupOperatorOverloads(opName, category, syntax, operatorList);
 
-      // combine the two.
-      List<SqlOperator> both = Lists.newArrayList();
-      both.addAll(optiqOps);
-      both.addAll(drillOps);
+      if(!operatorList.isEmpty()){
+        return;
+      }
 
-      return both;
+      List<SqlOperator> drillOps = opMap.get(opName.getSimple());
+      if(drillOps != null){
+        operatorList.addAll(drillOps);
+      }
 
     } else {
-      return inner.lookupOperatorOverloads(opName, category, syntax);
+      inner.lookupOperatorOverloads(opName, category, syntax, operatorList);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
deleted file mode 100644
index d0a6a19..0000000
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
+++ /dev/null
@@ -1,82 +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.planner.sql;
-
-import java.io.Reader;
-import java.util.List;
-
-import org.eigenbase.sql.SqlCall;
-import org.eigenbase.sql.SqlFunction;
-import org.eigenbase.sql.SqlFunctionCategory;
-import org.eigenbase.sql.SqlIdentifier;
-import org.eigenbase.sql.SqlLiteral;
-import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.SqlOperator;
-import org.eigenbase.sql.SqlOperatorTable;
-import org.eigenbase.sql.SqlSyntax;
-import org.eigenbase.sql.parser.SqlParserImplFactory;
-import org.eigenbase.sql.parser.SqlParserPos;
-import org.eigenbase.sql.parser.impl.SqlParserImpl;
-
-public class DrillParserFactory implements SqlParserImplFactory {
-  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillParserFactory.class);
-
-  private final SqlOperatorTable table;
-
-  public DrillParserFactory(SqlOperatorTable table) {
-    super();
-    this.table = table;
-  }
-
-  public DrillParser getParser(Reader stream) {
-    return new DrillParser(stream);
-  }
-
-  public class DrillParser extends SqlParserImpl {
-
-    public DrillParser(Reader stream) {
-      super(stream);
-    }
-
-    protected SqlCall createCall(SqlIdentifier funName, SqlParserPos pos, SqlFunctionCategory funcType,
-        SqlLiteral functionQualifier, SqlNode[] operands) {
-      SqlOperator fun = null;
-
-      // First, try a half-hearted resolution as a builtin function.
-      // If we find one, use it; this will guarantee that we
-      // preserve the correct syntax (i.e. don't quote builtin function
-      // / name when regenerating SQL).
-      if (funName.isSimple()) {
-        List<SqlOperator> list = table.lookupOperatorOverloads(funName, null, SqlSyntax.FUNCTION);
-        if (list.size() == 1) {
-          fun = list.get(0);
-        }
-      }
-
-      // Otherwise, just create a placeholder function. Later, during
-      // validation, it will be resolved into a real function reference.
-      if (fun == null) {
-        fun = new SqlFunction(funName, null, null, null, null, funcType);
-      }
-
-      return fun.createCall(functionQualifier, pos, operands);
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index d5ad1fc..e48bc5b 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -28,212 +28,61 @@ import net.hydromatic.optiq.tools.RelConversionException;
 import net.hydromatic.optiq.tools.RuleSet;
 import net.hydromatic.optiq.tools.ValidationException;
 
-import org.apache.drill.common.config.DrillConfig;
-import org.apache.drill.common.logical.LogicalPlan;
-import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
-import org.apache.drill.exec.client.QuerySubmitter;
-import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.planner.logical.DrillImplementor;
-import org.apache.drill.exec.planner.logical.DrillParseContext;
-import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
-import org.apache.drill.exec.planner.logical.DrillScreenRel;
-import org.apache.drill.exec.planner.logical.DrillStoreRel;
-import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
-import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
-import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.store.StoragePluginRegistry.DrillSchemaFactory;
+import org.apache.drill.exec.planner.sql.handlers.DefaultSqlHandler;
+import org.apache.drill.exec.planner.sql.handlers.ExplainHandler;
+import org.apache.drill.exec.planner.sql.handlers.SetOptionHandler;
+import org.apache.drill.exec.planner.sql.handlers.SqlHandler;
 import org.eigenbase.rel.RelCollationTraitDef;
-import org.eigenbase.rel.RelNode;
 import org.eigenbase.relopt.ConventionTraitDef;
-import org.eigenbase.relopt.RelOptUtil;
 import org.eigenbase.relopt.RelTraitDef;
-import org.eigenbase.relopt.RelTraitSet;
-import org.eigenbase.sql.SqlExplain;
-import org.eigenbase.sql.SqlExplainLevel;
-import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlLiteral;
 import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.parser.SqlParseException;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.hive12.common.base.Preconditions;
+import org.eigenbase.sql.parser.impl.SqlParserImpl;
+import org.eigenbase.sql2rel.StandardConvertletTable;
 
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
 
   private final Planner planner;
   private final static RuleSet[] RULES = new RuleSet[]{DrillRuleSets.DRILL_BASIC_RULES, DrillRuleSets.DRILL_PHYSICAL_MEM};
-  private final static int LOGICAL_RULES = 0;
-  private final static int PHYSICAL_MEM_RULES = 1;
-  
-  public DrillSqlWorker(DrillSchemaFactory schemaFactory, FunctionImplementationRegistry registry) throws Exception {
+  public final static int LOGICAL_RULES = 0;
+  public final static int PHYSICAL_MEM_RULES = 1;
+  private final QueryContext context;
+
+  public DrillSqlWorker(QueryContext context) throws Exception {
     final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+
     traitDefs.add(ConventionTraitDef.INSTANCE);
-    traitDefs.add(DrillDistributionTraitDef.INSTANCE);    
+    traitDefs.add(DrillDistributionTraitDef.INSTANCE);
     traitDefs.add(RelCollationTraitDef.INSTANCE);
-    
-    DrillOperatorTable table = new DrillOperatorTable(registry);
-    DrillParserFactory factory = new DrillParserFactory(table);
-    this.planner = Frameworks.getPlanner(Lex.MYSQL, factory, schemaFactory, table, traitDefs, RULES);
-//    this.planner = Frameworks.getPlanner(Lex.MYSQL, SqlParserImpl.FACTORY, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
-  }
-  
-  public class RelResult{
-    final ResultMode mode;
-    final RelNode node;
-    public RelResult(ResultMode mode, RelNode node) {
-      super();
-      this.mode = mode;
-      this.node = node;
-    }
-    
-    public ResultMode getMode() {
-      return this.mode;
-    }
-  }
+    this.context = context;
+    DrillOperatorTable table = new DrillOperatorTable(context.getFunctionRegistry());
+    this.planner = Frameworks.getPlanner(Lex.MYSQL, SqlParserImpl.FACTORY, context.getNewDefaultSchema(), table, traitDefs, StandardConvertletTable.INSTANCE, RULES);
 
-  /*
-   * Given a SQL string, return the logical DrillRel tree, plus mode (execute, or EXPLAIN mode).  
-   */
-  public RelResult getLogicalRel(String sql) throws SqlParseException, ValidationException, RelConversionException{
-    if(logger.isDebugEnabled()) {
-      logger.debug("SQL : " + sql);
-    }
-
-    // Call optiq to parse the SQL string. 
-    SqlNode sqlNode = planner.parse(sql);  
-    ResultMode resultMode = ResultMode.EXEC;
-    
-    //Process EXPLAIN
-    if(sqlNode.getKind() == SqlKind.EXPLAIN){
-      SqlExplain explain = (SqlExplain) sqlNode;
-      sqlNode = explain.operand(0);
-      SqlExplain.Depth depth = (SqlExplain.Depth) explain.getDepth();
-      switch(depth){
-      case LOGICAL:
-        resultMode = ResultMode.LOGICAL;
-        break;
-      case PHYSICAL:
-        resultMode = ResultMode.PHYSICAL;
-        break;
-      default:
-      }
-    }
-    
-    // Call optiq to validate SqlNode tree and convert it to RelNode tree. 
-    SqlNode validatedNode = planner.validate(sqlNode);
-    RelNode relNode = planner.convert(validatedNode);
-    
-    //Debug
-    if(logger.isDebugEnabled()) {
-      logger.debug("RelNode tree : " + RelOptUtil.toString(relNode, SqlExplainLevel.ALL_ATTRIBUTES));
-    }
-    
-    // Call optiq to transform RelNode into Drill Logical RelNode tree. 
-    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, relNode.getTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
-    if(convertedRelNode instanceof DrillStoreRel){
-      throw new UnsupportedOperationException();
-    }else{
-      convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
-    }
-    
-    //Debug
-    if(logger.isDebugEnabled()) {
-      logger.debug("Drill LogicalRel tree : " + RelOptUtil.toString(convertedRelNode, SqlExplainLevel.ALL_ATTRIBUTES));
-    }
-    
-    return new RelResult(resultMode, convertedRelNode);
   }
-  
-  /*
-   * Given a Drill LogicalRel tree, return Drill Logical Plan. 
-   * @param relResult :  RelResult whose node is the root of Drill logicalrel tree.
-   */
-  public LogicalPlan getLogicalPlan(RelResult relResult) throws SqlParseException, ValidationException, RelConversionException{
-    RelNode logicalRelRoot = relResult.node;
-    
-    Preconditions.checkArgument(logicalRelRoot.getConvention() == DrillRel.DRILL_LOGICAL);
-    
-    DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), relResult.mode);
-    implementor.go( (DrillRel) logicalRelRoot);
-    planner.close();
-    planner.reset();
-    return implementor.getPlan();    
-  }
-
-  /*
-   * Given a SQL string, return the Drill logical plan.
-   */
-  public LogicalPlan getLogicalPlan(String sql) throws SqlParseException, ValidationException, RelConversionException{
-    RelResult result = getLogicalRel(sql);
 
-    RelNode convertedRelNode = planner.transform(LOGICAL_RULES, result.node.getTraitSet().plus(DrillRel.DRILL_LOGICAL), result.node);
-    if(convertedRelNode instanceof DrillStoreRel){
-      throw new UnsupportedOperationException();
-    }else{
-      convertedRelNode = new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
-    }
-    DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), result.mode);
-    implementor.go( (DrillRel) convertedRelNode);
-    planner.close();
-    planner.reset();
-    return implementor.getPlan();    
-  }
+  public PhysicalPlan getPlan(String sql) throws SqlParseException, ValidationException, RelConversionException, IOException{
+    SqlNode sqlNode = planner.parse(sql);
 
-  /*
-   * Given a Drill LogicalRel tree, return Drill Physical plan.
-   * @param relResult : RelResult whose node is the root of Drill logicalrel tree.
-   * @param qcontext  : QueryContext used by PhysicalPlanCreator. 
-   */
-  public PhysicalPlan getPhysicalPlan(RelResult relResult, QueryContext qcontext) throws SqlParseException, ValidationException, RelConversionException, IOException {
-    RelNode logicalRelRoot = relResult.node;
-    
-    Preconditions.checkArgument(logicalRelRoot.getConvention() == DrillRel.DRILL_LOGICAL);
-    
-    RelTraitSet traits = logicalRelRoot.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);    
-    Prel phyRelNode = (Prel) planner.transform(PHYSICAL_MEM_RULES, traits, logicalRelRoot);
-    
-    //Debug
-    if(logger.isDebugEnabled()) {     
-      String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
-      logger.debug("Drill PhysicalRel tree: " + msg);
-    }
-    
-    PhysicalPlanCreator pplanCreator = new PhysicalPlanCreator(qcontext);
-    PhysicalPlan plan = pplanCreator.build(phyRelNode, true /* rebuild */);
-        
-    planner.close();
-    planner.reset();
-    return plan;
-  }
-  
-  /*
-   * Given a SQL string, return Drill physical plan. 
-   */
-  public PhysicalPlan getPhysicalPlan(String sql, QueryContext qcontext) throws SqlParseException, ValidationException, RelConversionException, IOException {
-    RelResult result = getLogicalRel(sql);
+    SqlHandler handler;
 
-    RelTraitSet traits = result.node.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);    
-    Prel phyRelNode = (Prel) planner.transform(PHYSICAL_MEM_RULES, traits, result.node);
-    
-    //Debug.
-    if(logger.isDebugEnabled()) {     
-      logger.debug("SQL : " + sql);
-      String msg = RelOptUtil.toString(phyRelNode, SqlExplainLevel.ALL_ATTRIBUTES);
-      logger.debug("Drill PhysicalRel tree: " + msg);      
+    // TODO: make this use path scanning or something similar.
+    switch(sqlNode.getKind()){
+    case EXPLAIN:
+      handler = new ExplainHandler(planner, context);
+      break;
+    case SET_OPTION:
+      handler = new SetOptionHandler(context);
+      break;
+    default:
+      handler = new DefaultSqlHandler(planner, context);
     }
-        
-    PhysicalPlanCreator pplanCreator = new PhysicalPlanCreator(qcontext);
-    PhysicalPlan plan = pplanCreator.build(phyRelNode, true /* rebuild */);
-        
-    planner.close();
-    planner.reset();
-    return plan;
 
+    return handler.getPlan(sqlNode);
   }
- 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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
new file mode 100644
index 0000000..a34a724
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/DefaultSqlHandler.java
@@ -0,0 +1,173 @@
+/**
+ * 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 java.io.IOException;
+import java.util.Collection;
+import java.util.List;
+
+import net.hydromatic.optiq.tools.Planner;
+import net.hydromatic.optiq.tools.RelConversionException;
+import net.hydromatic.optiq.tools.RuleSet;
+import net.hydromatic.optiq.tools.ValidationException;
+
+import org.apache.drill.common.logical.PlanProperties;
+import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.common.logical.PlanProperties.PlanPropertiesBuilder;
+import org.apache.drill.common.logical.PlanProperties.PlanType;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.AbstractPhysicalVisitor;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.logical.DrillRuleSets;
+import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.DrillStoreRel;
+import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
+import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
+import org.apache.drill.exec.planner.physical.PlanningSettings;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.sql.DrillSqlWorker;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptUtil;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.sql.SqlExplainLevel;
+import org.eigenbase.sql.SqlNode;
+
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.google.common.base.Preconditions;
+import com.google.hive12.common.collect.Lists;
+
+public class DefaultSqlHandler implements SqlHandler{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DefaultSqlHandler.class);
+
+
+  protected final Planner planner;
+  protected final QueryContext context;
+
+
+  public DefaultSqlHandler(Planner planner, QueryContext context) {
+    super();
+    this.planner = planner;
+    this.context = context;
+  }
+
+  protected void log(String name, RelNode node){
+    if(logger.isDebugEnabled()){
+      logger.debug(name + " : \n" + RelOptUtil.toString(node, SqlExplainLevel.ALL_ATTRIBUTES));
+    }
+  }
+
+  protected void log(String name, PhysicalPlan plan) throws JsonProcessingException{
+    if(logger.isDebugEnabled()){
+      String planText = plan.unparse(context.getConfig().getMapper().writer());
+      logger.debug(name + " : \n" + planText);
+    }
+  }
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException {
+
+    SqlNode validated = validateNode(sqlNode);
+    RelNode rel = convertToRel(validated);
+    log("Optiq Logical", rel);
+    DrillRel drel = convertToDrel(rel);
+    log("Drill Logical", drel);
+    Prel prel = convertToPrel(drel);
+    log("Drill Physical", prel);
+    PhysicalOperator pop = convertToPop(prel);
+    PhysicalPlan plan = convertToPlan(pop);
+    log("Drill Plan", plan);
+
+    return plan;
+  }
+
+  protected SqlNode validateNode(SqlNode sqlNode) throws ValidationException, RelConversionException{
+    return planner.validate(sqlNode);
+  }
+
+  protected RelNode convertToRel(SqlNode node) throws RelConversionException{
+    return planner.convert(node);
+  }
+
+  protected DrillRel convertToDrel(RelNode relNode) throws RelConversionException{
+    RelNode convertedRelNode = planner.transform(DrillSqlWorker.LOGICAL_RULES, relNode.getTraitSet().plus(DrillRel.DRILL_LOGICAL), relNode);
+    if(convertedRelNode instanceof DrillStoreRel){
+      throw new UnsupportedOperationException();
+    }else{
+      return new DrillScreenRel(convertedRelNode.getCluster(), convertedRelNode.getTraitSet(), convertedRelNode);
+    }
+  }
+
+  protected Prel convertToPrel(RelNode drel) throws RelConversionException{
+    Preconditions.checkArgument(drel.getConvention() == DrillRel.DRILL_LOGICAL);
+    RelTraitSet traits = drel.getTraitSet().plus(Prel.DRILL_PHYSICAL).plus(DrillDistributionTrait.SINGLETON);
+    Prel phyRelNode = (Prel) planner.transform(DrillSqlWorker.PHYSICAL_MEM_RULES, traits, drel);
+    return phyRelNode;
+  }
+
+  protected PhysicalOperator convertToPop(Prel prel) throws IOException{
+
+    boolean singleMode = !context.getSession().isEnableExchanges();
+
+    if(singleMode) PlanningSettings.get().setSingleMode(true);
+    PhysicalPlanCreator creator = new PhysicalPlanCreator(context);
+    PhysicalOperator op =  prel.getPhysicalOperator(creator);
+
+    if(singleMode) PlanningSettings.get().setSingleMode(false);
+    return op;
+  }
+
+  protected PhysicalPlan convertToPlan(PhysicalOperator op){
+    PlanPropertiesBuilder propsBuilder = PlanProperties.builder();
+    propsBuilder.type(PlanType.APACHE_DRILL_PHYSICAL);
+    propsBuilder.version(1);
+    propsBuilder.resultMode(ResultMode.EXEC);
+    propsBuilder.generator(this.getClass().getSimpleName(), "");
+    return new PhysicalPlan(propsBuilder.build(), getPops(op));
+  }
+
+
+  public static List<PhysicalOperator> getPops(PhysicalOperator root){
+    List<PhysicalOperator> ops = Lists.newArrayList();
+    PopCollector c = new PopCollector();
+    root.accept(c, ops);
+    return ops;
+  }
+
+  private static class PopCollector extends AbstractPhysicalVisitor<Void, Collection<PhysicalOperator>, RuntimeException>{
+
+    @Override
+    public Void visitOp(PhysicalOperator op, Collection<PhysicalOperator> collection) throws RuntimeException {
+      collection.add(op);
+      for(PhysicalOperator o : op){
+        o.accept(this, collection);
+      }
+      return null;
+    }
+
+  }
+
+  public static <T> T unwrap(Object o, Class<T> clazz) throws RelConversionException{
+    if(clazz.isAssignableFrom(o.getClass())){
+      return (T) o;
+    }else{
+      throw new RelConversionException(String.format("Failure trying to treat %s as type %s.", o.getClass().getSimpleName(), clazz.getSimpleName()));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
new file mode 100644
index 0000000..404f624
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/ExplainHandler.java
@@ -0,0 +1,116 @@
+/**
+ * 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 java.io.IOException;
+
+import net.hydromatic.optiq.tools.Planner;
+import net.hydromatic.optiq.tools.RelConversionException;
+import net.hydromatic.optiq.tools.ValidationException;
+
+import org.apache.drill.common.logical.LogicalPlan;
+import org.apache.drill.common.logical.PlanProperties.Generator.ResultMode;
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.planner.logical.DrillImplementor;
+import org.apache.drill.exec.planner.logical.DrillParseContext;
+import org.apache.drill.exec.planner.logical.DrillRel;
+import org.apache.drill.exec.planner.physical.Prel;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.relopt.RelOptUtil;
+import org.eigenbase.sql.SqlExplain;
+import org.eigenbase.sql.SqlExplainLevel;
+import org.eigenbase.sql.SqlLiteral;
+import org.eigenbase.sql.SqlNode;
+
+public class ExplainHandler extends DefaultSqlHandler{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExplainHandler.class);
+
+  private ResultMode mode;
+
+  public ExplainHandler(Planner planner, QueryContext context) {
+    super(planner, context);
+  }
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode node) throws ValidationException, RelConversionException, IOException {
+    SqlExplain explain = unwrap(node, SqlExplain.class);
+    SqlNode sqlNode = rewrite(explain);
+    SqlNode validated = validateNode(sqlNode);
+    RelNode rel = convertToRel(validated);
+    DrillRel drel = convertToDrel(rel);
+
+    if(mode == ResultMode.LOGICAL){
+      LogicalExplain logicalResult = new LogicalExplain(drel);
+      return DirectPlan.createDirectPlan(context, logicalResult);
+    }
+
+    Prel prel = convertToPrel(drel);
+    PhysicalOperator pop = convertToPop(prel);
+    PhysicalPlan plan = convertToPlan(pop);
+    PhysicalExplain physicalResult = new PhysicalExplain(prel, plan);
+    return DirectPlan.createDirectPlan(context, physicalResult);
+  }
+
+  private SqlNode rewrite(SqlExplain node) {
+    SqlLiteral op = (SqlLiteral) node.operand(2);
+    SqlExplain.Depth depth = (SqlExplain.Depth) op.getValue();
+
+    switch(depth){
+    case LOGICAL:
+      mode = ResultMode.LOGICAL;
+      break;
+    case PHYSICAL:
+      mode = ResultMode.PHYSICAL;
+      break;
+    default:
+      throw new UnsupportedOperationException("Unknown depth " + depth);
+    }
+
+    return node.operand(0);
+  }
+
+
+  public class LogicalExplain{
+    public String text;
+    public String json;
+
+    public LogicalExplain(RelNode node){
+      this.text = RelOptUtil.toString(node, SqlExplainLevel.DIGEST_ATTRIBUTES);
+      DrillImplementor implementor = new DrillImplementor(new DrillParseContext(), ResultMode.LOGICAL);
+      implementor.go( (DrillRel) node);
+      LogicalPlan plan = implementor.getPlan();
+      this.json = plan.unparse(context.getConfig());
+    }
+  }
+
+  public class PhysicalExplain{
+    public String text;
+    public String json;
+
+    public PhysicalExplain(RelNode node, PhysicalPlan plan){
+      this.text = RelOptUtil.toString(node, SqlExplainLevel.ALL_ATTRIBUTES);
+      this.json = plan.unparse(context.getConfig().getMapper().writer());
+    }
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
new file mode 100644
index 0000000..28180d2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SetOptionHandler.java
@@ -0,0 +1,58 @@
+/**
+ * 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 java.io.IOException;
+
+import net.hydromatic.optiq.tools.RelConversionException;
+import net.hydromatic.optiq.tools.ValidationException;
+
+import org.apache.drill.exec.ops.QueryContext;
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.apache.drill.exec.planner.sql.DirectPlan;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.SqlSetOption;
+
+public class SetOptionHandler implements SqlHandler{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SetOptionHandler.class);
+
+  QueryContext context;
+
+
+  public SetOptionHandler(QueryContext context) {
+    super();
+    this.context = context;
+  }
+
+
+  @Override
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException {
+    SqlSetOption option = DefaultSqlHandler.unwrap(sqlNode, SqlSetOption.class);
+    String scope = option.getScope();
+    String name = option.getName();
+    SqlNode value = option.getValue();
+    if(name.equals("NO_EXCHANGES")){
+      context.getSession().enableExchanges(false);
+    }
+    return DirectPlan.createDirectPlan(context, true, "disabled exchanges.");
+
+  }
+
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SimpleCommandResult.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SimpleCommandResult.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SimpleCommandResult.java
new file mode 100644
index 0000000..6dd7e7b
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SimpleCommandResult.java
@@ -0,0 +1,32 @@
+/**
+ * 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;
+
+public class SimpleCommandResult {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SimpleCommandResult.class);
+
+  public boolean ok;
+  public String summary;
+
+  public SimpleCommandResult(boolean ok, String summary) {
+    super();
+    this.ok = ok;
+    this.summary = summary;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandler.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandler.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandler.java
new file mode 100644
index 0000000..b95ebc0
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/handlers/SqlHandler.java
@@ -0,0 +1,32 @@
+/**
+ * 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 java.io.IOException;
+
+import net.hydromatic.optiq.tools.RelConversionException;
+import net.hydromatic.optiq.tools.ValidationException;
+
+import org.apache.drill.exec.physical.PhysicalPlan;
+import org.eigenbase.sql.SqlNode;
+
+public interface SqlHandler {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SqlHandler.class);
+
+  public PhysicalPlan getPlan(SqlNode sqlNode) throws ValidationException, RelConversionException, IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
index 8515b0c..8bc746e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/types/RelDataTypeHolder.java
@@ -32,9 +32,9 @@ public class RelDataTypeHolder {
   List<RelDataTypeField> fields = Lists.newArrayList();
 
   private RelDataTypeFactory typeFactory;
-  
+
   public List<RelDataTypeField> getFieldList(RelDataTypeFactory typeFactory) {
-    
+
     addStarIfEmpty(typeFactory);
     return fields;
   }
@@ -45,10 +45,11 @@ public class RelDataTypeHolder {
   }
 
   private void addStarIfEmpty(RelDataTypeFactory typeFactory){
-//    RelDataTypeField starCol = getField(typeFactory, "*");
-//    if (fields.isEmpty()) fields.add(starCol);
+    if (fields.isEmpty()){
+      getField(typeFactory, "*");
+    }
   }
-  
+
   public RelDataTypeField getField(RelDataTypeFactory typeFactory, String fieldName) {
 
     /* First check if this field name exists in our field list */
@@ -72,12 +73,12 @@ public class RelDataTypeHolder {
     for(RelDataTypeField f : fields){
       fieldNames.add(f.getName());
     };
-    
+
     return fieldNames;
   }
-  
+
   public void setRelDataTypeFactory(RelDataTypeFactory typeFactory) {
     this.typeFactory = typeFactory;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
index e658dac..abe6308 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/MaterializedField.java
@@ -17,10 +17,13 @@
  */
 package org.apache.drill.exec.record;
 
+import java.util.Collections;
 import java.util.Iterator;
+import java.util.List;
 
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos.DataMode;
 import org.apache.drill.common.types.TypeProtos.MajorType;
@@ -29,21 +32,28 @@ import org.apache.drill.exec.proto.SchemaDefProtos.FieldDef;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart;
 import org.apache.drill.exec.proto.SchemaDefProtos.NamePart.Type;
 
+import com.beust.jcommander.internal.Lists;
+
 public class MaterializedField{
   private final FieldDef def;
 
   public MaterializedField(FieldDef def) {
     this.def = def;
   }
-  
+
   public static MaterializedField create(FieldDef def){
     return new MaterializedField(def);
   }
-  
+
   public MaterializedField clone(FieldReference ref){
     return create(ref, def.getMajorType());
   }
-  
+
+  public static MaterializedField create(String path, MajorType type){
+    SchemaPath p = SchemaPath.getSimplePath(path);
+    return create(p, type);
+  }
+
   public static MaterializedField create(SchemaPath path, MajorType type) {
     FieldDef.Builder b = FieldDef.newBuilder();
     b.setMajorType(type);
@@ -68,26 +78,43 @@ public class MaterializedField{
   public FieldDef getDef() {
     return def;
   }
-  
-  public String getName(){
-    StringBuilder sb = new StringBuilder();
-    boolean first = true;
-    for(NamePart np : def.getNameList()){
-      if(np.getType() == Type.ARRAY){
-        sb.append("[]");
+
+  public SchemaPath getAsSchemaPath(){
+    List<NamePart> nameList = def.getNameList();
+    Collections.reverse(Lists.newArrayList(nameList));
+    PathSegment seg = null;
+    for(NamePart p : nameList){
+      if(p.getType() == NamePart.Type.ARRAY){
+        throw new UnsupportedOperationException();
       }else{
-        if(first){
-          first = false;
-        }else{
-          sb.append(".");
-        }
-        sb.append(np.getName());
-        
+        seg = new NameSegment(p.getName(), seg);
       }
     }
-    return sb.toString();
+    if( !(seg instanceof NameSegment) ) throw new UnsupportedOperationException();
+    return new SchemaPath( (NameSegment) seg);
   }
 
+//  public String getName(){
+//    StringBuilder sb = new StringBuilder();
+//    boolean first = true;
+//    for(NamePart np : def.getNameList()){
+//      if(np.getType() == Type.ARRAY){
+//        sb.append("[]");
+//      }else{
+//        if(first){
+//          first = false;
+//        }else{
+//          sb.append(".");
+//        }
+//        sb.append('`');
+//        sb.append(np.getName());
+//        sb.append('`');
+//
+//      }
+//    }
+//    return sb.toString();
+//  }
+
   public int getWidth() {
     return def.getMajorType().getWidth();
   }
@@ -103,7 +130,7 @@ public class MaterializedField{
   public DataMode getDataMode() {
     return def.getMajorType().getMode();
   }
-  
+
   public MaterializedField getOtherNullableVersion(){
     MajorType mt = def.getMajorType();
     DataMode newDataMode = null;
@@ -119,14 +146,14 @@ public class MaterializedField{
     }
     return new MaterializedField(def.toBuilder().setMajorType(mt.toBuilder().setMode(newDataMode).build()).build());
   }
-  
+
   public Class<?> getValueClass() {
     return TypeHelper.getValueVectorClass(getType().getMinorType(), getDataMode());
   }
 
   public boolean matches(SchemaPath path) {
     Iterator<NamePart> iter = def.getNameList().iterator();
-    
+
     for (PathSegment p = path.getRootSegment();; p = p.getChild()) {
       if(p == null) break;
       if (!iter.hasNext()) return false;
@@ -139,14 +166,14 @@ public class MaterializedField{
         if (p.getNameSegment().getPath().equalsIgnoreCase(n.getName())) continue;
         return false;
       }
-      
+
     }
     // we've reviewed all path segments. confirm that we don't have any extra name parts.
     return !iter.hasNext();
   }
-  
-  
-  
+
+
+
   @Override
   public int hashCode() {
     final int prime = 31;
@@ -177,4 +204,7 @@ public class MaterializedField{
     return "MaterializedField [" + def.toString() + "]";
   }
 
+  public String toExpr(){
+    return this.getAsSchemaPath().toExpr();
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
index bbeae4f..ed450af 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/RecordBatchLoader.java
@@ -61,13 +61,14 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
 //    logger.debug("Loading record batch with def {} and data {}", def, buf);
     this.valueCount = def.getRecordCount();
     boolean schemaChanged = schema == null;
-
+//    logger.info("Load, ThreadID: {}", Thread.currentThread().getId(), new RuntimeException("For Stack Trace Only"));
+//    System.out.println("Load, ThreadId: " + Thread.currentThread().getId());
     Map<FieldDef, ValueVector> oldFields = Maps.newHashMap();
     for(VectorWrapper<?> w : container){
       ValueVector v = w.getValueVector();
       oldFields.put(v.getField().getDef(), v);
     }
-    
+
     VectorContainer newVectors = new VectorContainer();
 
     List<FieldMetadata> fields = def.getFieldList();
@@ -114,10 +115,10 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
   public TypedFieldId getValueVectorId(SchemaPath path) {
     return container.getValueVectorId(path);
   }
-  
-  
-  
-//  
+
+
+
+//
 //  @SuppressWarnings("unchecked")
 //  public <T extends ValueVector> T getValueVectorId(int fieldId, Class<?> clazz) {
 //    ValueVector v = container.get(fieldId);
@@ -138,7 +139,7 @@ public class RecordBatchLoader implements VectorAccessible, Iterable<VectorWrapp
   public VectorWrapper<?> getValueAccessorById(int fieldId, Class<?> clazz){
     return container.getValueAccessorById(fieldId, clazz);
   }
-  
+
   public WritableBatch getWritableBatch(){
     boolean isSV2 = (schema.getSelectionVectorMode() == BatchSchema.SelectionVectorMode.TWO_BYTE);
     return WritableBatch.getBatchNoHVWrap(valueCount, container, isSV2);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
index 8bfb616..17fdc1e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/record/VectorContainer.java
@@ -51,7 +51,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
   public void addHyperList(List<ValueVector> vectors) {
     addHyperList(vectors, true);
   }
-  
+
   public void addHyperList(List<ValueVector> vectors, boolean releasable) {
     schema = null;
     ValueVector[] vv = new ValueVector[vectors.size()];
@@ -66,7 +66,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
    * container have the same TypedFieldIds as the existing container, allowing interchangeability in generated code. In
    * the case of hyper vectors, this container actually doesn't do a full transfer, rather creating a clone vector
    * wrapper only.
-   * 
+   *
    * @param incoming
    *          The RecordBatch iterator the contains the batch we should take over.
    * @return A cloned vector container.
@@ -129,9 +129,12 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
   public TypedFieldId getValueVectorId(SchemaPath path) {
     for (int i = 0; i < wrappers.size(); i++) {
       VectorWrapper<?> va = wrappers.get(i);
-      if (va.getField().matches(path))
+      SchemaPath w = va.getField().getAsSchemaPath();
+      if (w.equals(path)){
         return new TypedFieldId(va.getField().getType(), i, va.isHyper());
+      }
     }
+
     if(path.getRootSegment().isNamed() && path.getRootSegment().getNameSegment().getPath().equals("_MAP") && path.getRootSegment().isLastPath()) throw new UnsupportedOperationException("Drill does not yet support map references.");
     return null;
   }
@@ -190,7 +193,7 @@ public class VectorContainer implements Iterable<VectorWrapper<?>>, VectorAccess
     Preconditions.checkState(recordCount != -1, "Record count not set for this vector container");
     return recordCount;
   }
-  
+
   public void zeroVectors(){
     for (VectorWrapper<?> w : wrappers) {
       w.clear();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
new file mode 100644
index 0000000..c344ce1
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/rpc/user/DrillUser.java
@@ -0,0 +1,36 @@
+/**
+ * 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.rpc.user;
+
+import java.io.IOException;
+
+import org.apache.hadoop.security.UserGroupInformation;
+
+public class DrillUser {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillUser.class);
+
+  private UserGroupInformation hadoopUser;
+
+  public DrillUser(String userName) throws IOException {
+    this.hadoopUser = UserGroupInformation.createProxyUser(userName, UserGroupInformation.getCurrentUser());
+  }
+
+  public UserGroupInformation getHadoopUser(){
+    return hadoopUser;
+  }
+}


[31/51] [abbrv] git commit: in merge receiver, handle case where no records are returned

Posted by ja...@apache.org.
in merge receiver, handle case where no records are returned


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

Branch: refs/heads/master
Commit: bfbc83f907d38d2bcee6e0e1c2b22242326d22a4
Parents: ca58de1
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 15:35:50 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../physical/impl/mergereceiver/MergingRecordBatch.java     | 9 +++++++--
 1 file changed, 7 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/bfbc83f9/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
index ee2244e..c5c77a6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/mergereceiver/MergingRecordBatch.java
@@ -146,6 +146,9 @@ public class MergingRecordBatch implements RecordBatch {
 
       // allocate the incoming record batch loaders
       senderCount = rawBatches.size();
+      if (senderCount == 0) {
+        return IterOutcome.NONE;
+      }
       incomingBatches = new RawFragmentBatch[senderCount];
       batchOffsets = new int[senderCount];
       batchLoaders = new RecordBatchLoader[senderCount];
@@ -634,8 +637,10 @@ public class MergingRecordBatch implements RecordBatch {
   @Override
   public void cleanup() {
     outgoingContainer.clear();
-    for(RecordBatchLoader rbl : batchLoaders){
-      rbl.clear();
+    if (batchLoaders != null) {
+      for(RecordBatchLoader rbl : batchLoaders){
+        rbl.clear();
+      }
     }
   }
 


[28/51] [abbrv] git commit: Add couple of optiq rule into logical plan phase. Add operator mapping for is not null, is true.

Posted by ja...@apache.org.
Add couple of optiq rule into logical plan phase. Add operator mapping for is not null, is true.


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

Branch: refs/heads/master
Commit: 2038bc629abb2a9620c8aade2be7bdf4d5f097a6
Parents: eeb8535
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Tue Apr 1 15:18:36 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../common/expression/FunctionCallFactory.java  |  4 +-
 .../exec/planner/logical/DrillRuleSets.java     | 61 +++++++++++++++-----
 2 files changed, 49 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2038bc62/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
index 03be80f..50fceda 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
@@ -58,7 +58,9 @@ public class FunctionCallFactory {
     opToFuncTable.put("<>", "not_equal");
     opToFuncTable.put(">=", "greater_than_or_equal_to");
     opToFuncTable.put("<=", "less_than_or_equal_to");
-
+    opToFuncTable.put("is not null", "isnotnull");
+    opToFuncTable.put("is true", "istrue");
+    
     opToFuncTable.put("!", "not");
     opToFuncTable.put("u-", "negative");
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/2038bc62/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 562a3d4..d17fce3 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
@@ -30,9 +30,21 @@ 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.eigenbase.rel.RelFactories;
+import org.eigenbase.rel.rules.MergeProjectRule;
 import org.eigenbase.rel.rules.PushFilterPastJoinRule;
 import org.eigenbase.rel.rules.PushFilterPastProjectRule;
 import org.eigenbase.rel.rules.PushJoinThroughJoinRule;
+import org.eigenbase.rel.rules.PushSortPastProjectRule;
+import org.eigenbase.rel.rules.ReduceAggregatesRule;
+import org.eigenbase.rel.rules.RemoveDistinctAggregateRule;
+import org.eigenbase.rel.rules.RemoveDistinctRule;
+import org.eigenbase.rel.rules.RemoveSortRule;
+import org.eigenbase.rel.rules.RemoveTrivialCalcRule;
+import org.eigenbase.rel.rules.RemoveTrivialProjectRule;
+import org.eigenbase.rel.rules.SwapJoinRule;
+import org.eigenbase.rel.rules.TableAccessRule;
+import org.eigenbase.rel.rules.UnionToDistinctRule;
 import org.eigenbase.relopt.RelOptRule;
 import org.eigenbase.relopt.volcano.AbstractConverter.ExpandConversionRule;
 
@@ -49,6 +61,27 @@ public class DrillRuleSets {
       PushJoinThroughJoinRule.LEFT, 
       // End supprot for WHERE style joins.
       
+      //Add back rules
+      
+      ExpandConversionRule.INSTANCE,
+//      SwapJoinRule.INSTANCE,
+      RemoveDistinctRule.INSTANCE,
+//      UnionToDistinctRule.INSTANCE,
+//      RemoveTrivialProjectRule.INSTANCE,
+//      RemoveTrivialCalcRule.INSTANCE,
+      RemoveSortRule.INSTANCE,
+
+//      TableAccessRule.INSTANCE, //
+      //MergeProjectRule.INSTANCE, //
+      new MergeProjectRule(true, RelFactories.DEFAULT_PROJECT_FACTORY),
+      RemoveDistinctAggregateRule.INSTANCE, //
+      ReduceAggregatesRule.INSTANCE, //
+//      SwapJoinRule.INSTANCE, //
+//      PushJoinThroughJoinRule.RIGHT, //
+//      PushJoinThroughJoinRule.LEFT, //
+      PushSortPastProjectRule.INSTANCE, //      
+
+      ////////////////////////////////
       DrillScanRule.INSTANCE,
       DrillFilterRule.INSTANCE,
       DrillProjectRule.INSTANCE,
@@ -57,9 +90,7 @@ public class DrillRuleSets {
       DrillLimitRule.INSTANCE,
       DrillSortRule.INSTANCE,
       DrillJoinRule.INSTANCE,
-      DrillUnionRule.INSTANCE
-      
-      
+      DrillUnionRule.INSTANCE      
 
       ));
   
@@ -85,21 +116,21 @@ public class DrillRuleSets {
       FilterPrule.INSTANCE,
       LimitPrule.INSTANCE
 
-//    ExpandConversionRule.instance,
-//    SwapJoinRule.instance,
-//    RemoveDistinctRule.instance,
-//    UnionToDistinctRule.instance,
-//    RemoveTrivialProjectRule.instance,
-//    RemoveTrivialCalcRule.instance,
+//    ExpandConversionRule.INSTANCE,
+//    SwapJoinRule.INSTANCE,
+//    RemoveDistinctRule.INSTANCE,
+//    UnionToDistinctRule.INSTANCE,
+//    RemoveTrivialProjectRule.INSTANCE,
+//    RemoveTrivialCalcRule.INSTANCE,
 //    RemoveSortRule.INSTANCE,
 //
-//    TableAccessRule.instance, //
-//    MergeProjectRule.instance, //
-//    PushFilterPastProjectRule.instance, //
+//    TableAccessRule.INSTANCE, //
+//    MergeProjectRule.INSTANCE, //
+//    PushFilterPastProjectRule.INSTANCE, //
 //    PushFilterPastJoinRule.FILTER_ON_JOIN, //
-//    RemoveDistinctAggregateRule.instance, //
-//    ReduceAggregatesRule.instance, //
-//    SwapJoinRule.instance, //
+//    RemoveDistinctAggregateRule.INSTANCE, //
+//    ReduceAggregatesRule.INSTANCE, //
+//    SwapJoinRule.INSTANCE, //
 //    PushJoinThroughJoinRule.RIGHT, //
 //    PushJoinThroughJoinRule.LEFT, //
 //    PushSortPastProjectRule.INSTANCE, //      


[49/51] [abbrv] git commit: AltSort: Replace sort with ExternalSort, Limit + sort with Limit + topN in the generated physical plan.

Posted by ja...@apache.org.
AltSort: Replace sort with ExternalSort, Limit + sort with Limit + topN in the generated physical plan.


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

Branch: refs/heads/master
Commit: ecd64102f7514113c22b5286e6258495b5ec2707
Parents: 4f98a4f
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Wed Apr 9 12:33:31 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../exec/planner/common/DrillLimitRelBase.java  |  9 +++
 .../exec/planner/logical/DrillRuleSets.java     |  4 +-
 .../physical/DrillDistributionTrait.java        |  2 +-
 .../physical/HashToRandomExchangePrel.java      | 12 +++
 .../exec/planner/physical/PushLimitToTopN.java  | 52 ++++++++++++
 .../physical/SingleMergeExchangePrel.java       | 18 ++++-
 .../drill/exec/planner/physical/SortPrel.java   | 10 ++-
 .../drill/exec/planner/physical/TopNPrel.java   | 78 ++++++++++++++++++
 .../org/apache/drill/TestAltSortQueries.java    | 83 ++++++++++++++++++++
 .../org/apache/drill/TestExampleQueries.java    |  2 +-
 .../drill/jdbc/test/TestJdbcDistQuery.java      |  2 +
 11 files changed, 267 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
index c64b79b..b62eb9e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillLimitRelBase.java
@@ -27,6 +27,7 @@ import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.torel.ConversionContext;
 import org.eigenbase.rel.InvalidRelException;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
 import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelTraitSet;
@@ -54,4 +55,12 @@ public abstract class DrillLimitRelBase extends SingleRel implements DrillRelNod
   public RexNode getFetch() {
     return this.fetch;
   }
+  
+  public RelWriter explainTerms(RelWriter pw) {
+    super.explainTerms(pw);
+    pw.itemIf("offset", offset, offset != null);
+    pw.itemIf("fetch", fetch, fetch != null);
+    return pw;
+  }  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/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 5bd8581..e5cc730 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
@@ -25,6 +25,7 @@ import org.apache.drill.exec.planner.physical.FilterPrule;
 import org.apache.drill.exec.planner.physical.LimitPrule;
 import org.apache.drill.exec.planner.physical.MergeJoinPrule;
 import org.apache.drill.exec.planner.physical.ProjectPrule;
+import org.apache.drill.exec.planner.physical.PushLimitToTopN;
 import org.apache.drill.exec.planner.physical.ScanPrule;
 import org.apache.drill.exec.planner.physical.ScreenPrule;
 import org.apache.drill.exec.planner.physical.SortConvertPrule;
@@ -114,7 +115,8 @@ public class DrillRuleSets {
       StreamAggPrule.INSTANCE,
       MergeJoinPrule.INSTANCE,
       FilterPrule.INSTANCE,
-      LimitPrule.INSTANCE
+      LimitPrule.INSTANCE,
+      PushLimitToTopN.INSTANCE
 
 //    ExpandConversionRule.INSTANCE,
 //    SwapJoinRule.INSTANCE,

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
index 018f548..8573fb2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
@@ -138,7 +138,7 @@ public class DrillDistributionTrait implements RelTrait {
     }
     
     public String toString() {
-      return new Integer(fieldId).toString();
+      return String.format("[$%s]", this.fieldId);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
index c2b6c68..82f21eb 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/HashToRandomExchangePrel.java
@@ -20,12 +20,15 @@ package org.apache.drill.exec.planner.physical;
 import java.io.IOException;
 import java.util.List;
 
+import net.hydromatic.linq4j.Ord;
+
 import org.apache.drill.exec.physical.base.PhysicalOperator;
 import org.apache.drill.exec.physical.config.HashToRandomExchange;
 import org.apache.drill.exec.physical.config.SelectionVectorRemover;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
 import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptCost;
@@ -74,4 +77,13 @@ public class HashToRandomExchangePrel extends SingleRel implements Prel {
     return this.fields;
   }
   
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    super.explainTerms(pw);
+      for (Ord<DistributionField> ord : Ord.zip(fields)) {
+        pw.item("dist" + ord.i, ord.e);
+      }
+    return pw;
+  }  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PushLimitToTopN.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PushLimitToTopN.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PushLimitToTopN.java
new file mode 100644
index 0000000..4c5cf33
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PushLimitToTopN.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.planner.physical;
+
+import org.apache.drill.exec.planner.logical.RelOptHelper;
+import org.eigenbase.relopt.RelOptRule;
+import org.eigenbase.relopt.RelOptRuleCall;
+import org.eigenbase.rex.RexLiteral;
+
+public class PushLimitToTopN  extends RelOptRule{
+  
+  public static final RelOptRule INSTANCE = new PushLimitToTopN();
+
+  private PushLimitToTopN() {
+    super(RelOptHelper.some(LimitPrel.class, RelOptHelper.some(SingleMergeExchangePrel.class, RelOptHelper.any(SortPrel.class))), "PushLimitToTopN");
+  }
+
+  @Override
+  public void onMatch(RelOptRuleCall call) {
+    final LimitPrel limit = (LimitPrel) call.rel(0);
+    final SingleMergeExchangePrel smex = (SingleMergeExchangePrel) call.rel(1);
+    final SortPrel sort = (SortPrel) call.rel(2);
+    
+    // First offset to include into results (inclusive). Null implies it is starting from offset 0
+    int offset = limit.getOffset() != null ? Math.max(0, RexLiteral.intValue(limit.getOffset())) : 0;
+    int fetch = limit.getFetch() != null?  Math.max(0, RexLiteral.intValue(limit.getFetch())) : 0;
+        
+    final TopNPrel topN = new TopNPrel(limit.getCluster(), sort.getTraitSet(), sort.getChild(), offset + fetch, sort.getCollation()); 
+    final LimitPrel newLimit = new LimitPrel(limit.getCluster(), limit.getTraitSet(), 
+        new SingleMergeExchangePrel(smex.getCluster(), smex.getTraitSet(), topN, sort.getCollation()), 
+        limit.getOffset(), limit.getFetch());
+    
+    call.transformTo(newLimit);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
index 115d623..0d41a71 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SingleMergeExchangePrel.java
@@ -21,6 +21,8 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
+import net.hydromatic.linq4j.Ord;
+
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.logical.data.Order.Ordering;
@@ -32,6 +34,7 @@ import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
 import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelFieldCollation;
 import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
 import org.eigenbase.rel.SingleRel;
 import org.eigenbase.relopt.RelOptCluster;
 import org.eigenbase.relopt.RelOptCost;
@@ -76,5 +79,18 @@ public class SingleMergeExchangePrel extends SingleRel implements Prel {
     creator.addPhysicalOperator(g);
     return g;    
   }
-    
+  
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    super.explainTerms(pw);
+    if (pw.nest()) {
+      pw.item("collation", collation);
+    } else {
+      for (Ord<RelFieldCollation> ord : Ord.zip(collation.getFieldCollations())) {
+        pw.item("sort" + ord.i, ord.e);
+      }
+    }
+    return pw;
+  }  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
index 06f6e8b..9c73d77 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/SortPrel.java
@@ -20,6 +20,7 @@ package org.apache.drill.exec.planner.physical;
 import java.io.IOException;
 
 import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.ExternalSort;
 import org.apache.drill.exec.physical.config.SingleMergeExchange;
 import org.apache.drill.exec.physical.config.Sort;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -27,6 +28,8 @@ import org.eigenbase.rel.RelCollation;
 import org.eigenbase.rel.RelNode;
 import org.eigenbase.rel.SortRel;
 import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTraitSet;
 import org.eigenbase.rex.RexNode;
 
@@ -52,7 +55,7 @@ public class SortPrel extends SortRel implements Prel {
       throw new UnsupportedOperationException();
     }
     
-    Sort g = new Sort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
+    Sort g = new ExternalSort(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false);
     
     creator.addPhysicalOperator(g);
     
@@ -68,5 +71,10 @@ public class SortPrel extends SortRel implements Prel {
     return new SortPrel(getCluster(), traitSet, newInput, newCollation);
   }
   
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice. 
+    return super.computeSelfCost(planner).multiplyBy(0.1);
+  }
   
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
new file mode 100644
index 0000000..7c811f7
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/TopNPrel.java
@@ -0,0 +1,78 @@
+/**
+ * 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.physical;
+
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.drill.exec.physical.base.PhysicalOperator;
+import org.apache.drill.exec.physical.config.TopN;
+import org.eigenbase.rel.RelCollation;
+import org.eigenbase.rel.RelNode;
+import org.eigenbase.rel.RelWriter;
+import org.eigenbase.rel.SingleRel;
+import org.eigenbase.relopt.RelOptCluster;
+import org.eigenbase.relopt.RelOptCost;
+import org.eigenbase.relopt.RelOptPlanner;
+import org.eigenbase.relopt.RelTraitSet;
+import org.eigenbase.rex.RexLiteral;
+import org.eigenbase.rex.RexNode;
+
+public class TopNPrel extends SingleRel implements Prel {
+
+  protected int limit;
+  protected final RelCollation collation;
+  
+  public TopNPrel(RelOptCluster cluster, RelTraitSet traitSet, RelNode child, int limit, RelCollation collation) {
+    super(cluster, traitSet, child);
+    this.limit = limit;
+    this.collation = collation;
+  }
+
+  @Override
+  public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
+    return new TopNPrel(getCluster(), traitSet, sole(inputs), this.limit, this.collation);
+  }
+
+  @Override
+  public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
+    Prel child = (Prel) this.getChild();
+    
+    PhysicalOperator childPOP = child.getPhysicalOperator(creator);
+     
+    TopN topN = new TopN(childPOP, PrelUtil.getOrdering(this.collation, getChild().getRowType()), false, this.limit);
+    
+    creator.addPhysicalOperator(topN);
+    
+    return topN;
+  }
+  
+  @Override
+  public RelOptCost computeSelfCost(RelOptPlanner planner) {
+    //We use multiplier 0.05 for TopN operator, and 0.1 for Sort, to make TopN a preferred choice. 
+    return super.computeSelfCost(planner).multiplyBy(0.05);
+  }
+
+  
+  @Override
+  public RelWriter explainTerms(RelWriter pw) {
+    return super.explainTerms(pw)
+        .item("limit", limit);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
new file mode 100644
index 0000000..0521fca
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestAltSortQueries.java
@@ -0,0 +1,83 @@
+/**
+ * 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;
+
+import org.apache.drill.common.util.TestTools;
+import org.apache.drill.exec.client.QuerySubmitter;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestRule;
+
+public class TestAltSortQueries {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestAltSortQueries.class);
+  
+  @Rule public TestRule TIMEOUT = TestTools.getTimeoutRule(10000000);
+
+  @Test
+  public void testOrderBy() throws Exception{
+    test("select R_REGIONKEY " +
+         "from dfs.`[WORKING_PATH]/../../sample-data/region.parquet` " +
+         "order by R_REGIONKEY");   
+  }  
+
+  @Test 
+  public void testOrderBySingleFile() throws Exception{
+    test("select R_REGIONKEY " +
+         "from dfs.`[WORKING_PATH]/../../sample-data/regionsSF/` " +
+         "order by R_REGIONKEY");   
+  }  
+    
+  @Test
+  public void testSelectWithLimit() throws Exception{
+    test("select employee_id,  first_name, last_name from cp.`employee.json` order by employee_id limit 5 ");
+  }
+
+  @Test
+  public void testSelectWithLimitOffset() throws Exception{
+    test("select employee_id,  first_name, last_name from cp.`employee.json` order by employee_id limit 5 offset 10 ");
+  }
+
+  @Test
+  public void testJoinWithLimit() throws Exception{
+    test("SELECT\n" + 
+        "  nations.N_NAME,\n" + 
+        "  regions.R_NAME\n" + 
+        "FROM\n" + 
+        "  dfs.`[WORKING_PATH]/../../sample-data/nation.parquet` nations\n" + 
+        "JOIN\n" + 
+        "  dfs.`[WORKING_PATH]/../../sample-data/region.parquet` regions\n" + 
+        "  on nations.N_REGIONKEY = regions.R_REGIONKEY" +
+        " order by regions.R_NAME, nations.N_NAME " + 
+        " limit 5");
+  }
+    
+  
+  private void test(String sql) throws Exception{
+    boolean good = false;
+    sql = sql.replace("[WORKING_PATH]", TestTools.getWorkingPath());
+    
+    try{
+      QuerySubmitter s = new QuerySubmitter();
+      s.submitQuery(null, sql, "sql", null, true, 1, "tsv");
+      good = true;
+    }finally{
+      if(!good) Thread.sleep(2000);
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/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 35c4707..4f179a0 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
@@ -31,7 +31,7 @@ public class TestExampleQueries {
   
   @Test
   public void testSelectWithLimit() throws Exception{
-    test("select * from cp.`employee.json` limit 5");
+    test("select employee_id,  first_name, last_name from cp.`employee.json` order by employee_id limit 5 offset 10");
   }
   
   @Test

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecd64102/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
----------------------------------------------------------------------
diff --git a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
index 9977285..caa388a 100644
--- a/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
+++ b/sqlparser/src/test/java/org/apache/drill/jdbc/test/TestJdbcDistQuery.java
@@ -139,6 +139,7 @@ public class TestJdbcDistQuery {
   }
 
   @Test
+  //NPE at ExternalSortBatch.java : 151
   public void testSortSingleFile() throws Exception{
     testQuery(String.format("select R_REGIONKEY "
         + "from dfs.`%s/../sample-data/regionsSF/` "
@@ -146,6 +147,7 @@ public class TestJdbcDistQuery {
   }
 
   @Test
+  //NPE at ExternalSortBatch.java : 151
   public void testSortMultiFile() throws Exception{
     testQuery(String.format("select R_REGIONKEY "
         + "from dfs.`%s/../sample-data/regionsMF/` "


[04/51] [abbrv] git commit: - Fixes for more recent optiq snapshot - Fix hash rule to insert xor rather than ^ - Update dates so they write correctly - Add DrillSqlOperatorTable that supports resolving calls from Drill Function registry - Add Hack SqlPar

Posted by ja...@apache.org.
- Fixes for more recent optiq snapshot
- Fix hash rule to insert xor rather than ^
- Update dates so they write correctly
- Add DrillSqlOperatorTable that supports resolving calls from Drill Function registry
- Add Hack SqlParserImplFactory to override default AbstractSqlParserImpl use of Std operator table
- Remove spaces from function names to make ExprParser work correctly
- Add VarBinary to Hive object inspectors
- Update ExpressionTreeMaterializer's visit function call to return NULLExpression instead of null so that FunctionCall constructor's ImmutableList.of() doesn't throw exception on missing cast expression.


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

Branch: refs/heads/master
Commit: 62a786676c12e9ae258ef4eb0bc8322cb1cc7463
Parents: 6b517da
Author: Jacques Nadeau <ja...@apache.org>
Authored: Sun Mar 30 14:52:50 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:08 2014 -0700

----------------------------------------------------------------------
 common/pom.xml                                  |   5 +
 .../expression/ExpressionStringBuilder.java     |  46 ++++++--
 .../common/expression/FunctionCallFactory.java  |  12 +-
 .../common/expression/ValueExpressions.java     |   7 +-
 .../src/main/codegen/data/HiveTypes.tdd         |   8 ++
 .../codegen/templates/ComparisonFunctions.java  |  10 +-
 .../templates/DateIntervalFunctions.java        |  20 ++--
 .../codegen/templates/ObjectInspectors.java     |  21 ++++
 .../exec/expr/ExpressionTreeMaterializer.java   |   3 +-
 .../drill/exec/expr/fn/DrillFuncHolder.java     |   2 +-
 .../exec/planner/logical/DrillRuleSets.java     |  13 +-
 .../drill/exec/planner/physical/PrelUtil.java   |   2 +-
 .../apache/drill/exec/planner/sql/Checker.java  |  47 ++++++++
 .../exec/planner/sql/DrillOperatorTable.java    |  50 +++++---
 .../exec/planner/sql/DrillParserFactory.java    |  82 +++++++++++++
 .../exec/planner/sql/DrillSqlAggOperator.java   |  78 ++++++++++++
 .../exec/planner/sql/DrillSqlOperator.java      | 118 +------------------
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   9 +-
 .../exec/planner/sql/DynamicReturnType.java     |  29 +++++
 .../drill/exec/planner/sql/DynamicType.java     |  35 ++++++
 .../drill/exec/planner/sql/FixedRange.java      |  46 ++++++++
 .../drill/exec/resolver/TypeCastRules.java      |   4 +-
 .../apache/drill/exec/store/AbstractSchema.java |   8 +-
 .../drill/exec/store/StoragePluginRegistry.java |  16 +--
 .../exec/store/dfs/FileSystemSchemaFactory.java |  10 +-
 .../java/org/apache/drill/BaseTestQuery.java    |   2 +-
 .../java/org/apache/drill/TestTpchQueries.java  |   2 +-
 .../src/test/resources/queries/tpch/01.sql      |   4 +-
 28 files changed, 493 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index afcf7a3..55d4dd4 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -98,6 +98,11 @@
       <version>3.4</version>
     </dependency>
 
+    <dependency>
+        <groupId>joda-time</groupId>
+        <artifactId>joda-time</artifactId>
+        <version>2.3</version>
+    </dependency>
 
   </dependencies>
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 9b3e91a..35835c7 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -17,20 +17,27 @@
  */
 package org.apache.drill.common.expression;
 
+import java.io.IOException;
+
 import org.apache.drill.common.expression.IfExpression.IfCondition;
 import org.apache.drill.common.expression.ValueExpressions.BooleanExpression;
+import org.apache.drill.common.expression.ValueExpressions.DateExpression;
 import org.apache.drill.common.expression.ValueExpressions.DoubleExpression;
 import org.apache.drill.common.expression.ValueExpressions.FloatExpression;
 import org.apache.drill.common.expression.ValueExpressions.IntExpression;
-import org.apache.drill.common.expression.ValueExpressions.LongExpression;
-import org.apache.drill.common.expression.ValueExpressions.DateExpression;
-import org.apache.drill.common.expression.ValueExpressions.IntervalYearExpression;
 import org.apache.drill.common.expression.ValueExpressions.IntervalDayExpression;
-import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
-import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.IntervalYearExpression;
+import org.apache.drill.common.expression.ValueExpressions.LongExpression;
 import org.apache.drill.common.expression.ValueExpressions.QuotedString;
+import org.apache.drill.common.expression.ValueExpressions.TimeExpression;
+import org.apache.drill.common.expression.ValueExpressions.TimeStampExpression;
 import org.apache.drill.common.expression.visitors.AbstractExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
+import org.joda.time.Period;
+import org.joda.time.format.DateTimeFormat;
+import org.joda.time.format.DateTimeFormatter;
+import org.joda.time.format.DateTimeFormatterBuilder;
+import org.joda.time.format.DateTimeParser;
 
 import com.google.common.collect.ImmutableList;
 
@@ -88,31 +95,56 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
 
   @Override
   public Void visitDateConstant(DateExpression lExpr, StringBuilder sb) throws RuntimeException {
+    sb.append("cast( ");
     sb.append(lExpr.getDate());
+    sb.append(" as DATE)");
     return null;
   }
 
   @Override
   public Void visitTimeConstant(TimeExpression lExpr, StringBuilder sb) throws RuntimeException {
+    sb.append("cast( ");
     sb.append(lExpr.getTime());
+    sb.append(" as TIME)");
     return null;
   }
 
+  static final DateTimeFormatter TIMESTAMP_FORMAT;
+  static {
+    DateTimeFormatter dateFormatter = DateTimeFormat.forPattern("yyyy-MM-dd");
+    DateTimeParser optionalTime = DateTimeFormat.forPattern(" HH:mm:ss").getParser();
+    DateTimeParser optionalSec = DateTimeFormat.forPattern(".SSS").getParser();
+
+    TIMESTAMP_FORMAT = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).toFormatter();
+
+  }
+  
   @Override
   public Void visitTimeStampConstant(TimeStampExpression lExpr, StringBuilder sb) throws RuntimeException {
+    sb.append("cast( \"");
+    try {
+      TIMESTAMP_FORMAT.printTo(sb, lExpr.getTimeStamp());
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
     sb.append(lExpr.getTimeStamp());
+    sb.append(" \" as TIMESTAMP)");
     return null;
   }
 
   @Override
   public Void visitIntervalYearConstant(IntervalYearExpression lExpr, StringBuilder sb) throws RuntimeException {
-    sb.append(lExpr.getIntervalYear());
+    sb.append("cast( \"");
+    sb.append(Period.years(lExpr.getIntervalYear()).toString());
+    sb.append(" \" as INTERVAL)");
     return null;
   }
 
   @Override
   public Void visitIntervalDayConstant(IntervalDayExpression lExpr, StringBuilder sb) throws RuntimeException {
-    sb.append(lExpr.getIntervalDay() + " " + lExpr.getIntervalMillis());
+    sb.append("cast( \"");
+    sb.append(Period.days(lExpr.getIntervalDay()).plusMillis(lExpr.getIntervalMillis()).toString());
+    sb.append(" \" as INTERVAL)");
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
index c6c7074..71fd8bd 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
@@ -48,14 +48,14 @@ public class FunctionCallFactory {
 
     opToFuncTable.put("||", "or");
     opToFuncTable.put("&&", "and");
-    opToFuncTable.put(">", "greater than");
-    opToFuncTable.put("<", "less than");
+    opToFuncTable.put(">", "greater_than");
+    opToFuncTable.put("<", "less_than");
     opToFuncTable.put("==", "equal");
     opToFuncTable.put("=", "equal");
-    opToFuncTable.put("!=", "not equal");
-    opToFuncTable.put("<>", "not equal");
-    opToFuncTable.put(">=", "greater than or equal to");
-    opToFuncTable.put("<=", "less than or equal to");
+    opToFuncTable.put("!=", "not_equal");
+    opToFuncTable.put("<>", "not_equal");
+    opToFuncTable.put(">=", "greater_than_or_equal_to");
+    opToFuncTable.put("<=", "less_than_or_equal_to");
 
     opToFuncTable.put("!", "not");
     opToFuncTable.put("u-", "negative");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 8ec4109..10cf152 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -420,12 +420,13 @@ public class ValueExpressions {
   public static class IntervalDayExpression extends LogicalExpressionBase {
 
     private static final MajorType INTERVALDAY_CONSTANT = Types.required(MinorType.INTERVALDAY);
-
+    private static final long MILLIS_IN_DAY = 1000 * 60 * 60 * 24;
+    
     private int days;
     private int millis;
-
+    
     public IntervalDayExpression(long intervalInMillis) {
-      this((int) intervalInMillis / (1000 * 60 * 60 * 24), (int) (intervalInMillis % (1000 * 60 * 60 * 24)), ExpressionPosition.UNKNOWN);
+      this((int) (intervalInMillis / MILLIS_IN_DAY), (int) (intervalInMillis % MILLIS_IN_DAY), ExpressionPosition.UNKNOWN);
     }
 
       public IntervalDayExpression(int days, int millis, ExpressionPosition pos) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/codegen/data/HiveTypes.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/HiveTypes.tdd b/exec/java-exec/src/main/codegen/data/HiveTypes.tdd
index ddc8041..c23f981 100644
--- a/exec/java-exec/src/main/codegen/data/HiveTypes.tdd
+++ b/exec/java-exec/src/main/codegen/data/HiveTypes.tdd
@@ -87,6 +87,14 @@
       javaType: "",
       minorType: "VAR16CHAR",
       holder: "Var16Char"
+    },
+    {
+      hiveType: "BINARY",
+      hiveOI: "BinaryObjectInspector",
+      serdeConstant: "BINARY_TYPE_NAME",
+      javaType: "",
+      minorType: "VARBINARY",
+      holder: "VarBinary"
     }
   ]
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java b/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
index f5ccc6f..9b0028c 100644
--- a/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/ComparisonFunctions.java
@@ -118,7 +118,7 @@ public class GCompare${left}${right}{
       }
   }
   
-  @FunctionTemplate(names = {"less than", "<"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  @FunctionTemplate(names = {"less_than", "<"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
   public static class LessThan${left}${right} implements DrillSimpleFunc {
 
       @Param ${left}Holder left;
@@ -149,7 +149,7 @@ public class GCompare${left}${right}{
       }
   }
   
-  @FunctionTemplate(names = {"less than or equal to", "<="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  @FunctionTemplate(names = {"less_than_or_equal_to", "<="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
   public static class LessThanE${left}${right} implements DrillSimpleFunc {
 
       @Param ${left}Holder left;
@@ -180,7 +180,7 @@ public class GCompare${left}${right}{
     }
   }
   
-  @FunctionTemplate(names = {"greater than", ">"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  @FunctionTemplate(names = {"greater_than", ">"}, scope = FunctionTemplate.FunctionScope.SIMPLE)
   public static class GreaterThan${left}${right} implements DrillSimpleFunc {
 
       @Param ${left}Holder left;
@@ -211,7 +211,7 @@ public class GCompare${left}${right}{
     }
   }
   
-  @FunctionTemplate(names = {"greater than or equal to", ">="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  @FunctionTemplate(names = {"greater_than_or_equal_to", ">="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
   public static class GreaterThanE${left}${right} implements DrillSimpleFunc {
 
       @Param ${left}Holder left;
@@ -273,7 +273,7 @@ public class GCompare${left}${right}{
       }
   }
   
-  @FunctionTemplate(names = {"not equal","<>","!="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
+  @FunctionTemplate(names = {"not_equal","<>","!="}, scope = FunctionTemplate.FunctionScope.SIMPLE)
   public static class NotEquals${left}${right} implements DrillSimpleFunc {
 
       @Param ${left}Holder left;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
index eb0e1fb..7afbcfc 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctions.java
@@ -53,7 +53,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "less than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "less_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class LessThan${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -69,7 +69,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "less than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "less_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class LessThanE${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -85,7 +85,7 @@ public class GCompare${type.name}Functions {
     }
   }
 
-  @FunctionTemplate(name = "greater than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "greater_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class GreaterThan${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -101,7 +101,7 @@ public class GCompare${type.name}Functions {
     }
   }
 
-  @FunctionTemplate(name = "greater than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "greater_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class GreaterThanE${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -132,7 +132,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "not equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "not_equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class NotEquals${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -211,7 +211,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "less than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "less_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class LessThan${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -234,7 +234,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "less than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "less_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class LessThanE${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -257,7 +257,7 @@ public class GCompare${type.name}Functions {
     }
   }
 
-  @FunctionTemplate(name = "greater than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "greater_than", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class GreaterThan${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -280,7 +280,7 @@ public class GCompare${type.name}Functions {
     }
   }
 
-  @FunctionTemplate(name = "greater than or equal to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "greater_than_or_equal_to", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class GreaterThanE${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;
@@ -326,7 +326,7 @@ public class GCompare${type.name}Functions {
       }
   }
 
-  @FunctionTemplate(name = "not equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(name = "not_equal", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class NotEquals${type.name} implements DrillSimpleFunc {
 
       @Param ${type.name}Holder left;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/codegen/templates/ObjectInspectors.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/ObjectInspectors.java b/exec/java-exec/src/main/codegen/templates/ObjectInspectors.java
index 57a7bda..cf96cbc 100644
--- a/exec/java-exec/src/main/codegen/templates/ObjectInspectors.java
+++ b/exec/java-exec/src/main/codegen/templates/ObjectInspectors.java
@@ -65,6 +65,27 @@ public class Drill${entry.holder}ObjectInspector extends AbstractPrimitiveObject
     else
     return ((NullableVar16CharHolder)o).toString();
   }
+<#elseif entry.minorType == "VARBINARY">  
+@Override
+public org.apache.hadoop.io.BytesWritable getPrimitiveWritableObject(Object o) {
+  throw new UnsupportedOperationException();
+}
+
+@Override
+public byte[] getPrimitiveJavaObject(Object o) {
+  if (o instanceof VarBinaryHolder){
+    VarBinaryHolder h = (VarBinaryHolder)o;
+    byte[] buf = new byte[h.end-h.start];
+    h.buffer.getBytes(h.start, buf, 0, h.end-h.start);
+    return buf;
+  }else{
+    NullableVarBinaryHolder h = (NullableVarBinaryHolder)o;
+    byte[] buf = new byte[h.end-h.start];
+    h.buffer.getBytes(h.start, buf, 0, h.end-h.start);
+    return buf;
+    
+  }
+}
 <#elseif entry.minorType == "BIT">
   @Override
   public boolean get(Object o) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index 608c2ff..d65ff78 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -107,6 +107,7 @@ public class ExpressionTreeMaterializer {
       List<LogicalExpression> args = Lists.newArrayList();
       for (int i = 0; i < call.args.size(); ++i) {
         LogicalExpression newExpr = call.args.get(i).accept(this, registry);
+        assert newExpr != null : String.format("Materialization of %s return a null expression.", call.args.get(i));
         args.add(newExpr);
       }
 
@@ -150,7 +151,7 @@ public class ExpressionTreeMaterializer {
 
             if (matchedCastFuncHolder == null) {
               logFunctionResolutionError(errorCollector, castCall);
-              return null;
+              return NullExpression.INSTANCE;
             }
 
             argsWithCast.add(new DrillFuncHolderExpr(call.getName(), matchedCastFuncHolder, castArgs, ExpressionPosition.UNKNOWN));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
index 47b8408..e93a768 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/DrillFuncHolder.java
@@ -180,7 +180,7 @@ public abstract class DrillFuncHolder {
     return this.parameters[i].type;
   }
 
-  public int getParmSize() {
+  public int getParamCount() {
     return this.parameters.length;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/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 9ab5308..562a3d4 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
@@ -30,8 +30,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.eigenbase.rel.rules.PushFilterPastJoinRule;
+import org.eigenbase.rel.rules.PushFilterPastProjectRule;
+import org.eigenbase.rel.rules.PushJoinThroughJoinRule;
 import org.eigenbase.relopt.RelOptRule;
-import org.eigenbase.relopt.volcano.AbstractConverter;
 import org.eigenbase.relopt.volcano.AbstractConverter.ExpandConversionRule;
 
 import com.google.common.collect.ImmutableSet;
@@ -40,6 +42,12 @@ public class DrillRuleSets {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillRuleSets.class);
 
   public static final RuleSet DRILL_BASIC_RULES = new DrillRuleSet(ImmutableSet.of( //
+      // Add support for WHERE style joins.
+      PushFilterPastProjectRule.INSTANCE,
+      PushFilterPastJoinRule.FILTER_ON_JOIN,
+      PushJoinThroughJoinRule.RIGHT, 
+      PushJoinThroughJoinRule.LEFT, 
+      // End supprot for WHERE style joins.
       
       DrillScanRule.INSTANCE,
       DrillFilterRule.INSTANCE,
@@ -50,6 +58,9 @@ public class DrillRuleSets {
       DrillSortRule.INSTANCE,
       DrillJoinRule.INSTANCE,
       DrillUnionRule.INSTANCE
+      
+      
+
       ));
   
   public static final RuleSet DRILL_PHYSICAL_MEM = new DrillRuleSet(ImmutableSet.of( //

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
index 22e8090..45e7523 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/PrelUtil.java
@@ -63,7 +63,7 @@ public class PrelUtil {
       fr = new FieldReference(childFields.get(fields.get(i).getFieldId()), ExpressionPosition.UNKNOWN);      
       FunctionCall func2 = new FunctionCall("hash",  ImmutableList.of((LogicalExpression)fr), ExpressionPosition.UNKNOWN);
       
-      func = new FunctionCall("^", ImmutableList.of((LogicalExpression)func, (LogicalExpression)func2), ExpressionPosition.UNKNOWN);
+      func = new FunctionCall("xor", ImmutableList.of((LogicalExpression)func, (LogicalExpression)func2), ExpressionPosition.UNKNOWN);
     }
     
     return func;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.java
new file mode 100644
index 0000000..a22da17
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/Checker.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;
+
+import org.eigenbase.sql.SqlCallBinding;
+import org.eigenbase.sql.SqlOperandCountRange;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.sql.type.SqlOperandTypeChecker;
+
+class Checker implements SqlOperandTypeChecker {
+  private SqlOperandCountRange range;
+
+  public Checker(int size) {
+    range = new FixedRange(size);
+  }
+
+  @Override
+  public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
+    return true;
+  }
+
+  @Override
+  public SqlOperandCountRange getOperandCountRange() {
+    return range;
+  }
+
+  @Override
+  public String getAllowedSignatures(SqlOperator op, String opName) {
+    return opName + "(Drill - Opaque)";
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
index f20ca99..0dbbc56 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillOperatorTable.java
@@ -17,8 +17,10 @@
  */
 package org.apache.drill.exec.planner.sql;
 
+import java.util.Collection;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 import org.apache.drill.exec.expr.fn.DrillFuncHolder;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -31,44 +33,58 @@ import org.eigenbase.sql.fun.SqlStdOperatorTable;
 
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.Lists;
+import com.google.hive12.common.collect.Sets;
 
-public class DrillOperatorTable extends SqlStdOperatorTable{
+public class DrillOperatorTable extends SqlStdOperatorTable {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOperatorTable.class);
 
   private static final SqlOperatorTable inner = SqlStdOperatorTable.instance();
   private List<SqlOperator> operators;
   private ArrayListMultimap<String, SqlOperator> opMap = ArrayListMultimap.create();
-  
-  public DrillOperatorTable(FunctionImplementationRegistry registry){
+
+  public DrillOperatorTable(FunctionImplementationRegistry registry) {
     operators = Lists.newArrayList();
     operators.addAll(inner.getOperatorList());
-    
-    for(Map.Entry<String, DrillFuncHolder> function : registry.getDrillRegistry().getMethods().entries()){
-      SqlOperator op = new DrillSqlOperator(function.getKey(), function.getValue().getParmSize());
-      operators.add(op);
-      opMap.put(function.getKey(), op);
+
+    for (Map.Entry<String, Collection<DrillFuncHolder>> function : registry.getDrillRegistry().getMethods().asMap().entrySet()) {
+      Set<Integer> argCounts = Sets.newHashSet();
+      String name = function.getKey().toUpperCase();
+      for (DrillFuncHolder f : function.getValue()) {
+        if (argCounts.add(f.getParamCount())) {
+          SqlOperator op = null;
+          if (f.isAggregating()) {
+            op = new DrillSqlAggOperator(name, f.getParamCount());
+          } else {
+            op = new DrillSqlOperator(name, f.getParamCount());
+          }
+          operators.add(op);
+          opMap.put(function.getKey(), op);
+        }
+      }
     }
-    
+
     // TODO: add hive functions.
   }
-  
+
   @Override
   public List<SqlOperator> lookupOperatorOverloads(SqlIdentifier opName, SqlFunctionCategory category, SqlSyntax syntax) {
-    if(syntax == SqlSyntax.FUNCTION){
+    if (syntax == SqlSyntax.FUNCTION) {
       List<SqlOperator> drillOps = opMap.get(opName.getSimple());
-      if(drillOps == null || drillOps.isEmpty()) return inner.lookupOperatorOverloads(opName, category, syntax);
-      
+      if (drillOps == null || drillOps.isEmpty())
+        return inner.lookupOperatorOverloads(opName, category, syntax);
+
       List<SqlOperator> optiqOps = inner.lookupOperatorOverloads(opName, category, syntax);
-      if(optiqOps.isEmpty()) return drillOps;
-      
+      if (optiqOps.isEmpty())
+        return drillOps;
+
       // combine the two.
       List<SqlOperator> both = Lists.newArrayList();
       both.addAll(optiqOps);
       both.addAll(drillOps);
-      
+
       return both;
 
-    }else{
+    } else {
       return inner.lookupOperatorOverloads(opName, category, syntax);
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
new file mode 100644
index 0000000..d0a6a19
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillParserFactory.java
@@ -0,0 +1,82 @@
+/**
+ * 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;
+
+import java.io.Reader;
+import java.util.List;
+
+import org.eigenbase.sql.SqlCall;
+import org.eigenbase.sql.SqlFunction;
+import org.eigenbase.sql.SqlFunctionCategory;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlLiteral;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.SqlOperator;
+import org.eigenbase.sql.SqlOperatorTable;
+import org.eigenbase.sql.SqlSyntax;
+import org.eigenbase.sql.parser.SqlParserImplFactory;
+import org.eigenbase.sql.parser.SqlParserPos;
+import org.eigenbase.sql.parser.impl.SqlParserImpl;
+
+public class DrillParserFactory implements SqlParserImplFactory {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillParserFactory.class);
+
+  private final SqlOperatorTable table;
+
+  public DrillParserFactory(SqlOperatorTable table) {
+    super();
+    this.table = table;
+  }
+
+  public DrillParser getParser(Reader stream) {
+    return new DrillParser(stream);
+  }
+
+  public class DrillParser extends SqlParserImpl {
+
+    public DrillParser(Reader stream) {
+      super(stream);
+    }
+
+    protected SqlCall createCall(SqlIdentifier funName, SqlParserPos pos, SqlFunctionCategory funcType,
+        SqlLiteral functionQualifier, SqlNode[] operands) {
+      SqlOperator fun = null;
+
+      // First, try a half-hearted resolution as a builtin function.
+      // If we find one, use it; this will guarantee that we
+      // preserve the correct syntax (i.e. don't quote builtin function
+      // / name when regenerating SQL).
+      if (funName.isSimple()) {
+        List<SqlOperator> list = table.lookupOperatorOverloads(funName, null, SqlSyntax.FUNCTION);
+        if (list.size() == 1) {
+          fun = list.get(0);
+        }
+      }
+
+      // Otherwise, just create a placeholder function. Later, during
+      // validation, it will be resolved into a real function reference.
+      if (fun == null) {
+        fun = new SqlFunction(funName, null, null, null, null, funcType);
+      }
+
+      return fun.createCall(functionQualifier, pos, operands);
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
new file mode 100644
index 0000000..f18559f
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlAggOperator.java
@@ -0,0 +1,78 @@
+/**
+ * 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;
+
+import java.util.List;
+
+import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
+import org.apache.drill.exec.planner.types.RelDataTypeHolder;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.sql.SqlAggFunction;
+import org.eigenbase.sql.SqlCall;
+import org.eigenbase.sql.SqlCallBinding;
+import org.eigenbase.sql.SqlFunctionCategory;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlKind;
+import org.eigenbase.sql.SqlLiteral;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.SqlOperandCountRange;
+import org.eigenbase.sql.SqlOperatorBinding;
+import org.eigenbase.sql.SqlSyntax;
+import org.eigenbase.sql.SqlWriter;
+import org.eigenbase.sql.parser.SqlParserPos;
+import org.eigenbase.sql.type.SqlOperandTypeChecker;
+import org.eigenbase.sql.type.SqlOperandTypeInference;
+import org.eigenbase.sql.type.SqlReturnTypeInference;
+import org.eigenbase.sql.type.SqlTypeName;
+import org.eigenbase.sql.util.SqlBasicVisitor.ArgHandler;
+import org.eigenbase.sql.util.SqlVisitor;
+import org.eigenbase.sql.validate.SqlMonotonicity;
+import org.eigenbase.sql.validate.SqlValidator;
+import org.eigenbase.sql.validate.SqlValidatorScope;
+
+import com.google.hive12.common.collect.ImmutableList;
+
+public class DrillSqlAggOperator extends SqlAggFunction {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlAggOperator.class);
+
+  
+  DrillSqlAggOperator(String name, int argCount) {
+    super(name, SqlKind.OTHER_FUNCTION, DynamicReturnType.INSTANCE, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
+  }
+
+  @Override
+  public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
+    return getAny(validator.getTypeFactory());
+  }
+
+  private RelDataType getAny(RelDataTypeFactory factory){
+    return factory.createSqlType(SqlTypeName.ANY);
+//    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
+  }
+  
+  @Override
+  public List<RelDataType> getParameterTypes(RelDataTypeFactory typeFactory) {
+    return ImmutableList.of(typeFactory.createSqlType(SqlTypeName.ANY));
+  }
+
+  @Override
+  public RelDataType getReturnType(RelDataTypeFactory typeFactory) {
+    return getAny(typeFactory);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
index f09e1a1..4531914 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlOperator.java
@@ -18,138 +18,26 @@
 
 package org.apache.drill.exec.planner.sql;
 
-import java.util.Collection;
-import java.util.List;
-
-import org.apache.drill.exec.expr.fn.DrillFuncHolder;
-import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
-import org.apache.drill.exec.planner.types.RelDataTypeHolder;
 import org.eigenbase.reltype.RelDataType;
-import org.eigenbase.reltype.RelDataTypeFactory;
-import org.eigenbase.reltype.RelProtoDataType;
 import org.eigenbase.sql.SqlCall;
-import org.eigenbase.sql.SqlCallBinding;
 import org.eigenbase.sql.SqlFunction;
 import org.eigenbase.sql.SqlFunctionCategory;
-import org.eigenbase.sql.SqlIdentifier;
 import org.eigenbase.sql.SqlKind;
-import org.eigenbase.sql.SqlOperandCountRange;
-import org.eigenbase.sql.SqlOperator;
-import org.eigenbase.sql.SqlSyntax;
-import org.eigenbase.sql.type.ExplicitReturnTypeInference;
-import org.eigenbase.sql.type.SqlOperandCountRanges;
-import org.eigenbase.sql.type.SqlOperandTypeChecker;
 import org.eigenbase.sql.type.SqlTypeName;
 import org.eigenbase.sql.validate.SqlValidator;
 import org.eigenbase.sql.validate.SqlValidatorScope;
 
-import com.google.hive12.common.collect.Lists;
-
 public class DrillSqlOperator extends SqlFunction {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlOperator.class);
 
   DrillSqlOperator(String name, int argCount) {
-    super(name, SqlKind.OTHER_FUNCTION, DYNAMIC_RETURN, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
-  }
-
-  private static final DynamicReturnType DYNAMIC_RETURN = new DynamicReturnType();
-
-
-  @Override
-  public SqlIdentifier getSqlIdentifier() {
-    return super.getSqlIdentifier();
-  }
-
-  @Override
-  public SqlIdentifier getNameAsId() {
-    return super.getNameAsId();
-  }
-
-  @Override
-  public List<RelDataType> getParamTypes() {
-    return super.getParamTypes();
-  }
-
-  @Override
-  public SqlFunctionCategory getFunctionType() {
-    return super.getFunctionType();
-  }
-
-  @Override
-  public boolean isQuantifierAllowed() {
-    return super.isQuantifierAllowed();
+    super(name, SqlKind.OTHER_FUNCTION, DynamicReturnType.INSTANCE, null, new Checker(argCount), SqlFunctionCategory.USER_DEFINED_FUNCTION);
   }
 
   @Override
   public RelDataType deriveType(SqlValidator validator, SqlValidatorScope scope, SqlCall call) {
-    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), validator.getTypeFactory());
-//    return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
-    //return super.deriveType(validator, scope, call);
-  }
-
-  
-  private static class DynamicReturnType extends ExplicitReturnTypeInference {
-    public DynamicReturnType() {
-      super(new DynamicType());
-    }
+    return validator.getTypeFactory().createSqlType(SqlTypeName.ANY);
+//    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), validator.getTypeFactory());
   }
-  private static class DynamicType implements RelProtoDataType {
-
-    @Override
-    public RelDataType apply(RelDataTypeFactory factory) {
-      return factory.createSqlType(SqlTypeName.ANY);
-      //return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
-    }
-
-  }
-
-  private static class FixedRange implements SqlOperandCountRange{
-
-    private final int size;
-    
-    public FixedRange(int size) {
-      super();
-      this.size = size;
-    }
-
-    @Override
-    public boolean isValidCount(int count) {
-      return count == size;
-    }
 
-    @Override
-    public int getMin() {
-      return size;
-    }
-
-    @Override
-    public int getMax() {
-      return size;
-    }
-    
-  }
-  
-  private static class Checker implements SqlOperandTypeChecker {
-    private SqlOperandCountRange range;
-
-    public Checker(int size) {
-      range = new FixedRange(size);
-    }
-
-    @Override
-    public boolean checkOperandTypes(SqlCallBinding callBinding, boolean throwOnFailure) {
-      return true;
-    }
-
-    @Override
-    public SqlOperandCountRange getOperandCountRange() {
-      return range;
-    }
-
-    @Override
-    public String getAllowedSignatures(SqlOperator op, String opName) {
-      return opName + "(Drill - Opaque)";
-    }
-
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
index 676a1e3..30eb605 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DrillSqlWorker.java
@@ -35,17 +35,16 @@ import org.apache.drill.exec.client.QuerySubmitter;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.physical.PhysicalPlan;
-import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.logical.DrillImplementor;
 import org.apache.drill.exec.planner.logical.DrillParseContext;
 import org.apache.drill.exec.planner.logical.DrillRel;
 import org.apache.drill.exec.planner.logical.DrillRuleSets;
 import org.apache.drill.exec.planner.logical.DrillScreenRel;
+import org.apache.drill.exec.planner.logical.DrillStoreRel;
 import org.apache.drill.exec.planner.physical.DrillDistributionTrait;
 import org.apache.drill.exec.planner.physical.DrillDistributionTraitDef;
 import org.apache.drill.exec.planner.physical.PhysicalPlanCreator;
 import org.apache.drill.exec.planner.physical.Prel;
-import org.apache.drill.exec.planner.physical.DrillDistributionTrait.DistributionField;
 import org.apache.drill.exec.store.StoragePluginRegistry.DrillSchemaFactory;
 import org.eigenbase.rel.RelCollationTraitDef;
 import org.eigenbase.rel.RelNode;
@@ -58,11 +57,9 @@ import org.eigenbase.sql.SqlExplainLevel;
 import org.eigenbase.sql.SqlKind;
 import org.eigenbase.sql.SqlLiteral;
 import org.eigenbase.sql.SqlNode;
-import org.eigenbase.sql.fun.SqlStdOperatorTable;
 import org.eigenbase.sql.parser.SqlParseException;
 
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableList;
 
 public class DrillSqlWorker {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillSqlWorker.class);
@@ -78,7 +75,9 @@ public class DrillSqlWorker {
     traitDefs.add(DrillDistributionTraitDef.INSTANCE);    
     traitDefs.add(RelCollationTraitDef.INSTANCE);
     
-    this.planner = Frameworks.getPlanner(Lex.MYSQL, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
+    DrillOperatorTable table = new DrillOperatorTable(registry);
+    DrillParserFactory factory = new DrillParserFactory(table);
+    this.planner = Frameworks.getPlanner(Lex.MYSQL, factory, schemaFactory, table, traitDefs, RULES);
 //    this.planner = Frameworks.getPlanner(Lex.MYSQL, SqlParserImpl.FACTORY, schemaFactory, SqlStdOperatorTable.instance(), traitDefs, RULES);
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicReturnType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicReturnType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicReturnType.java
new file mode 100644
index 0000000..ee13330
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicReturnType.java
@@ -0,0 +1,29 @@
+/**
+ * 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;
+
+import org.eigenbase.sql.type.ExplicitReturnTypeInference;
+
+class DynamicReturnType extends ExplicitReturnTypeInference {
+  
+  public static final DynamicReturnType INSTANCE = new DynamicReturnType();
+  
+  public DynamicReturnType() {
+    super(new DynamicType());
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicType.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicType.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicType.java
new file mode 100644
index 0000000..5b65565
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/DynamicType.java
@@ -0,0 +1,35 @@
+/**
+ * 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;
+
+import org.apache.drill.exec.planner.types.RelDataTypeDrillImpl;
+import org.apache.drill.exec.planner.types.RelDataTypeHolder;
+import org.eigenbase.reltype.RelDataType;
+import org.eigenbase.reltype.RelDataTypeFactory;
+import org.eigenbase.reltype.RelProtoDataType;
+import org.eigenbase.sql.type.SqlTypeName;
+
+class DynamicType implements RelProtoDataType {
+
+  @Override
+  public RelDataType apply(RelDataTypeFactory factory) {
+    return factory.createSqlType(SqlTypeName.ANY);
+//    return new RelDataTypeDrillImpl(new RelDataTypeHolder(), factory);
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/FixedRange.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/FixedRange.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/FixedRange.java
new file mode 100644
index 0000000..d68b8f2
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/FixedRange.java
@@ -0,0 +1,46 @@
+/**
+ * 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;
+
+import org.eigenbase.sql.SqlOperandCountRange;
+
+class FixedRange implements SqlOperandCountRange{
+
+  private final int size;
+  
+  public FixedRange(int size) {
+    super();
+    this.size = size;
+  }
+
+  @Override
+  public boolean isValidCount(int count) {
+    return count == size;
+  }
+
+  @Override
+  public int getMin() {
+    return size;
+  }
+
+  @Override
+  public int getMax() {
+    return size;
+  }
+  
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
index 98f8127..ed89301 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/resolver/TypeCastRules.java
@@ -578,11 +578,11 @@ public class TypeCastRules {
   public static int getCost(FunctionCall call, DrillFuncHolder holder) {
     int cost = 0;
 
-    if (call.args.size() != holder.getParmSize()) {
+    if (call.args.size() != holder.getParamCount()) {
       return -1;
     }
       
-    for (int i = 0; i < holder.getParmSize(); i++) {
+    for (int i = 0; i < holder.getParamCount(); i++) {
       MajorType argType = call.args.get(i).getMajorType();
       MajorType parmType = holder.getParmMajorType(i);
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/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 5b8ccff..6aad394 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
@@ -23,9 +23,9 @@ import java.util.Set;
 
 import net.hydromatic.linq4j.expressions.DefaultExpression;
 import net.hydromatic.linq4j.expressions.Expression;
+import net.hydromatic.optiq.Function;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.TableFunction;
 
 import org.apache.drill.exec.planner.logical.DrillTable;
 
@@ -55,16 +55,14 @@ public abstract class AbstractSchema implements Schema{
   }
 
   @Override
-  public Collection<TableFunction> getTableFunctions(String name) {
+  public Collection<Function> getFunctions(String name) {
     return Collections.emptyList();
   }
 
   @Override
-  public Set<String> getTableFunctionNames() {
+  public Set<String> getFunctionNames() {
     return Collections.emptySet();
   }
-  
-  
 
   @Override
   public Schema getSubSchema(String name) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/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 ce15341..b9d8f33 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,10 +31,10 @@ import java.util.Set;
 import net.hydromatic.linq4j.expressions.DefaultExpression;
 import net.hydromatic.linq4j.expressions.Expression;
 import net.hydromatic.linq4j.function.Function1;
+import net.hydromatic.optiq.Function;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 import net.hydromatic.optiq.Table;
-import net.hydromatic.optiq.TableFunction;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
@@ -221,13 +221,13 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public Collection<TableFunction> getTableFunctions(String name) {
-      return inner.getTableFunctions(name);
+    public Collection<Function> getFunctions(String name) {
+      return inner.getFunctions(name);
     }
 
     @Override
-    public Set<String> getTableFunctionNames() {
-      return inner.getTableFunctionNames();
+    public Set<String> getFunctionNames() {
+      return inner.getFunctionNames();
     }
 
     @Override
@@ -274,12 +274,12 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public Collection<TableFunction> getTableFunctions(String name) {
+    public Collection<Function> getFunctions(String name) {
       return Collections.emptyList();
     }
 
     @Override
-    public Set<String> getTableFunctionNames() {
+    public Set<String> getFunctionNames() {
       return Collections.emptySet();
     }
 
@@ -311,7 +311,7 @@ public class StoragePluginRegistry implements Iterable<Map.Entry<String, Storage
     }
 
     @Override
-    public void add(String name, TableFunction table) {
+    public void add(String name, Function table) {
       throw new UnsupportedOperationException();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
index fbea81c..b150cc1 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/store/dfs/FileSystemSchemaFactory.java
@@ -22,9 +22,9 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import net.hydromatic.optiq.Function;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
-import net.hydromatic.optiq.TableFunction;
 
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
@@ -87,13 +87,13 @@ public class FileSystemSchemaFactory implements SchemaFactory{
     }
 
     @Override
-    public Collection<TableFunction> getTableFunctions(String name) {
-      return defaultSchema.getTableFunctions(name);
+    public Collection<Function> getFunctions(String name) {
+      return defaultSchema.getFunctions(name);
     }
 
     @Override
-    public Set<String> getTableFunctionNames() {
-      return defaultSchema.getTableFunctionNames();
+    public Set<String> getFunctionNames() {
+      return defaultSchema.getFunctionNames();
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/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 f724ca4..0e17f8e 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
@@ -32,7 +32,7 @@ import com.google.common.io.Resources;
 public class BaseTestQuery {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(BaseTestQuery.class);
   
-  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(10000);
+  @Rule public final TestRule TIMEOUT = TestTools.getTimeoutRule(20000);
   
   protected void test(String sql) throws Exception{
     boolean good = false;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java b/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
index 8643fa3..fada2f0 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/TestTpchQueries.java
@@ -36,7 +36,7 @@ public class TestTpchQueries extends BaseTestQuery{
   }
 
   @Test
-  @Ignore // DRILL-436
+  @Ignore
   public void tpch03() throws Exception{
     testSqlFromFile("queries/tpch/03.sql");
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/62a78667/exec/java-exec/src/test/resources/queries/tpch/01.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/01.sql b/exec/java-exec/src/test/resources/queries/tpch/01.sql
index b107339..f33416f 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/01.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/01.sql
@@ -13,8 +13,8 @@ select
   count(*) as count_order
 from
   cp.`tpch/lineitem.parquet`
--- where
---  l_shipdate <= date '1998-12-01' - interval '120' day (3)
+where
+  l_shipdate <= date '1998-12-01' - interval '120' day (3)
 group by
   l_returnflag,
   l_linestatus


[46/51] [abbrv] git commit: Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes

Posted by ja...@apache.org.
Move to Optiq 0.6
Also includes:
-improve exception catching
-move schema path parsing to Antlr
-close zookeeper connection on if client created
-enhance BaseTestQuery and have other query tests utilize it
-Various test fixes for better memory release.  still needs client allocator to be closed.
-refactor DrillSqlWorker and create multiple SqlHandlers
-Add PojoRecordReader and DirectPlan capabilities
-Update Antlr to use same quoting rules as SQL: single quote for quoted strings, back ticks for identifiers
-Move back to old Sorts until bugs are fixed
-Refector SelectionVector management within Prels
-Add support for NO_EXCHANGES option
-Extract SchemaFactories to use Optiq's new Schema handling capabilities
-Add basic handling of cancel in UserServer
-Remove output requirement from Project
-Add start of usercredentials to User communication


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

Branch: refs/heads/master
Commit: ecaa838fe7194c197884f0a1b6fb6f8a9683cede
Parents: ba0a639
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Apr 14 07:37:02 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:12 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/parser/ExprLexer.g  |  25 +-
 .../drill/common/expression/parser/ExprParser.g |  20 +-
 .../apache/drill/common/config/DrillConfig.java |  48 +-
 .../expression/ExpressionStringBuilder.java     |  64 ++-
 .../drill/common/expression/FieldReference.java |  31 +-
 .../common/expression/LogicalExpression.java    |   2 +
 .../drill/common/expression/PathSegment.java    | 110 ++--
 .../drill/common/expression/SchemaPath.java     | 179 ++++---
 .../common/expression/ValueExpressions.java     |  56 +-
 .../drill/common/logical/data/Project.java      |  22 +-
 .../org/apache/drill/common/util/TestTools.java |  46 +-
 .../common/expression/parser/TreeTest.java      | 102 +++-
 .../exec/store/hbase/HBaseRecordReader.java     |  40 +-
 .../exec/store/hbase/HBaseSchemaFactory.java    |   6 +-
 .../exec/store/hbase/HBaseStoragePlugin.java    |   7 +-
 distribution/src/resources/drill-env.sh         |   4 +-
 .../apache/drill/exec/client/DrillClient.java   |  46 +-
 .../org/apache/drill/exec/client/DumpCat.java   |  12 +-
 .../exec/client/PrintingResultsListener.java    | 100 ++++
 .../drill/exec/client/QuerySubmitter.java       | 184 +++----
 .../apache/drill/exec/compile/MergeAdapter.java |  42 +-
 .../org/apache/drill/exec/memory/Accountor.java |  17 +-
 .../drill/exec/memory/AtomicRemainder.java      |  15 +-
 .../apache/drill/exec/ops/FragmentContext.java  |  15 +-
 .../org/apache/drill/exec/ops/Multitimer.java   |  42 ++
 .../org/apache/drill/exec/ops/QueryContext.java |  59 ++-
 .../org/apache/drill/exec/ops/QuerySetup.java   |  26 +
 .../drill/exec/physical/impl/OutputMutator.java |   4 +
 .../drill/exec/physical/impl/ScanBatch.java     |  12 +-
 .../impl/TopN/PriorityQueueTemplate.java        |  10 +-
 .../physical/impl/common/HashTableTemplate.java | 125 +++--
 .../OrderedPartitionRecordBatch.java            |  42 +-
 .../partitionsender/OutgoingRecordBatch.java    |  14 +-
 .../impl/project/ProjectRecordBatch.java        |  53 +-
 .../exec/physical/impl/sort/SortBatch.java      |  40 +-
 .../impl/sort/SortRecordBatchBuilder.java       |  34 +-
 .../IteratorValidatorBatchIterator.java         |  35 +-
 .../physical/impl/xsort/ExternalSortBatch.java  |  12 +-
 .../planner/common/DrillProjectRelBase.java     |   4 +-
 .../exec/planner/logical/DrillProjectRel.java   |   4 +-
 .../exec/planner/logical/DrillRuleSets.java     |  27 +-
 .../drill/exec/planner/logical/ExprHelper.java  |   8 +-
 .../physical/DrillDistributionTrait.java        |  35 +-
 .../drill/exec/planner/physical/FilterPrel.java |  15 +-
 .../physical/HashToRandomExchangePrel.java      |  28 +-
 .../drill/exec/planner/physical/LimitPrel.java  |  14 +-
 .../exec/planner/physical/MergeJoinPrel.java    |  57 +-
 .../planner/physical/PhysicalPlanCreator.java   |  24 +-
 .../exec/planner/physical/PlanningSettings.java |  57 ++
 .../drill/exec/planner/physical/PrelUtil.java   |  39 +-
 .../exec/planner/physical/ProjectPrel.java      |  18 +-
 .../drill/exec/planner/physical/ScanPrel.java   |  11 +-
 .../drill/exec/planner/physical/ScreenPrel.java |  18 +-
 .../physical/SingleMergeExchangePrel.java       |  30 +-
 .../drill/exec/planner/physical/SortPrel.java   |  26 +-
 .../exec/planner/physical/StreamAggPrel.java    |  19 +-
 .../drill/exec/planner/physical/TopNPrel.java   |  16 +-
 .../planner/physical/UnionExchangePrel.java     |  20 +-
 .../drill/exec/planner/sql/DirectPlan.java      |  63 +++
 .../exec/planner/sql/DrillOperatorTable.java    |  26 +-
 .../exec/planner/sql/DrillParserFactory.java    |  82 ---
 .../drill/exec/planner/sql/DrillSqlWorker.java  | 213 ++------
 .../planner/sql/handlers/DefaultSqlHandler.java | 173 ++++++
 .../planner/sql/handlers/ExplainHandler.java    | 116 ++++
 .../planner/sql/handlers/SetOptionHandler.java  |  58 ++
 .../sql/handlers/SimpleCommandResult.java       |  32 ++
 .../exec/planner/sql/handlers/SqlHandler.java   |  32 ++
 .../exec/planner/types/RelDataTypeHolder.java   |  17 +-
 .../drill/exec/record/MaterializedField.java    |  80 ++-
 .../drill/exec/record/RecordBatchLoader.java    |  15 +-
 .../drill/exec/record/VectorContainer.java      |  11 +-
 .../apache/drill/exec/rpc/user/DrillUser.java   |  36 ++
 .../apache/drill/exec/rpc/user/UserServer.java  |  29 +-
 .../apache/drill/exec/rpc/user/UserSession.java |  80 +++
 .../org/apache/drill/exec/schema/Field.java     |   5 +-
 .../drill/exec/server/DrillbitContext.java      |  30 +-
 .../apache/drill/exec/store/SchemaFactory.java  |   7 +-
 .../apache/drill/exec/store/StoragePlugin.java  |   9 +-
 .../drill/exec/store/StoragePluginRegistry.java | 162 +-----
 .../drill/exec/store/dfs/FileSystemPlugin.java  |  24 +-
 .../exec/store/dfs/FileSystemSchemaFactory.java |  16 +-
 .../exec/store/direct/DirectBatchCreator.java   |  37 ++
 .../exec/store/direct/DirectGroupScan.java      |  81 +++
 .../drill/exec/store/direct/DirectSubScan.java  |  37 ++
 .../exec/store/easy/json/JSONRecordReader.java  |   7 +-
 .../drill/exec/store/hive/HiveRecordReader.java |   6 +-
 .../exec/store/hive/HiveStoragePlugin.java      |   9 +-
 .../store/hive/schema/HiveSchemaFactory.java    |  24 +-
 .../exec/store/ischema/EmptyVectorSet.java      |  72 +--
 .../store/ischema/InfoSchemaStoragePlugin.java  |  21 +-
 .../drill/exec/store/ischema/OptiqProvider.java |  24 +-
 .../drill/exec/store/mock/MockRecordReader.java |  12 +-
 .../exec/store/mock/MockStorageEngine.java      |   9 +-
 .../exec/store/parquet/ParquetRecordReader.java |   6 +-
 .../drill/exec/store/pojo/AbstractWriter.java   |  63 +++
 .../drill/exec/store/pojo/PojoRecordReader.java | 131 +++++
 .../drill/exec/store/pojo/PojoWriter.java       |  29 +
 .../apache/drill/exec/store/pojo/Writers.java   | 127 +++++
 .../apache/drill/exec/util/BatchPrinter.java    |   6 +-
 .../org/apache/drill/exec/util/VectorUtil.java  |   8 +-
 .../apache/drill/exec/work/foreman/Foreman.java | 158 ++----
 .../apache/drill/exec/work/user/UserWorker.java |  13 +-
 .../java/org/apache/drill/BaseTestQuery.java    | 152 ++++--
 .../java/org/apache/drill/PlanningBase.java     | 114 ++++
 .../org/apache/drill/TestAltSortQueries.java    |  48 +-
 .../org/apache/drill/TestExampleQueries.java    |  52 +-
 .../org/apache/drill/TestTpchDistributed.java   | 154 ++++++
 .../java/org/apache/drill/TestTpchExplain.java  | 155 ++++++
 .../java/org/apache/drill/TestTpchPlanning.java | 150 ++++++
 .../java/org/apache/drill/TestTpchQueries.java  | 158 ------
 .../org/apache/drill/TestTpchSingleMode.java    | 157 ++++++
 .../drill/exec/cache/TestVectorCache.java       |   2 +-
 .../exec/physical/impl/SimpleRootExec.java      |  15 +-
 .../physical/impl/TestBroadcastExchange.java    |   8 +-
 .../exec/physical/impl/TestCastFunctions.java   | 151 +++---
 .../physical/impl/TestCastVarCharToBigInt.java  |   5 +
 .../impl/TestDistributedFragmentRun.java        |  10 +-
 .../physical/impl/TestExtractFunctions.java     |   5 +
 .../physical/impl/TestHashToRandomExchange.java |   1 +
 .../exec/physical/impl/TestOptiqPlans.java      |  13 +-
 .../physical/impl/TestSimpleFragmentRun.java    |  13 +-
 .../exec/physical/impl/TestUnionExchange.java   |   1 +
 .../exec/physical/impl/TopN/TestSimpleTopN.java |  10 +-
 .../drill/exec/physical/impl/agg/TestAgg.java   |  34 +-
 .../exec/physical/impl/join/TestMergeJoin.java  |  20 +-
 .../impl/join/TestMergeJoinMulCondition.java    |  10 +-
 .../impl/mergereceiver/TestMergingReceiver.java |  14 +-
 .../TestOrderedPartitionExchange.java           |   3 +
 .../impl/xsort/TestSimpleExternalSort.java      |   8 +-
 .../drill/exec/record/vector/TestDateTypes.java |  30 ++
 .../drill/exec/record/vector/TestLoad.java      |   4 +-
 .../drill/exec/store/TestOrphanSchema.java      |  22 +-
 .../drill/exec/store/TestOutputMutator.java     |  15 +
 .../drill/exec/store/ischema/OrphanSchema.java  |  33 +-
 .../exec/store/ischema/TestOrphanSchema.java    |  49 +-
 .../exec/store/ischema/TestTableProvider.java   |  57 +-
 .../exec/store/json/JSONRecordReaderTest.java   |  17 +-
 .../store/parquet/ParquetRecordReaderTest.java  | 131 ++---
 .../store/parquet/ParquetResultListener.java    |  30 +-
 .../store/parquet/TestParquetPhysicalPlan.java  |   2 +-
 .../exec/work/batch/TestSpoolingBuffer.java     |   1 +
 .../src/test/resources/agg/hashagg/q6.json      |   4 +-
 .../src/test/resources/agg/hashagg/q7_1.json    |   6 +-
 .../src/test/resources/agg/hashagg/q7_2.json    |   6 +-
 .../src/test/resources/agg/hashagg/q8.json      |   4 +-
 .../src/test/resources/agg/hashagg/q8_1.json    |   4 +-
 .../functions/cast/testCastBigInt.json          |   2 +-
 .../functions/cast/testCastFloat4.json          |   4 +-
 .../functions/cast/testCastFloat8.json          |   4 +-
 .../resources/functions/cast/testCastInt.json   |   2 +-
 .../functions/cast/testCastNested.json          |   2 +-
 .../functions/cast/testCastNumException.json    |   2 +-
 .../functions/cast/testICastConstant.json       |  36 +-
 .../functions/simple_math_functions.json        |   8 +-
 .../functions/string/testCharLength.json        |  12 +-
 .../resources/functions/string/testConcat.json  |   8 +-
 .../resources/functions/string/testLeft.json    |  14 +-
 .../resources/functions/string/testLike.json    |   8 +-
 .../resources/functions/string/testLower.json   |   6 +-
 .../resources/functions/string/testLpad.json    |  20 +-
 .../resources/functions/string/testLtrim.json   |  12 +-
 .../functions/string/testPosition.json          |  16 +-
 .../functions/string/testRegexpReplace.json     |   6 +-
 .../resources/functions/string/testReplace.json |  12 +-
 .../resources/functions/string/testRight.json   |  14 +-
 .../resources/functions/string/testRpad.json    |  20 +-
 .../resources/functions/string/testRtrim.json   |  12 +-
 .../resources/functions/string/testSimilar.json |   8 +-
 .../resources/functions/string/testUpper.json   |   6 +-
 .../test/resources/join/mj_multi_condition.json |  12 +-
 .../test/resources/logical_string_filter.json   |   2 +-
 .../src/test/resources/simple_plan.json         |  12 +-
 pom.xml                                         |   8 +-
 .../org/apache/drill/exec/proto/BitControl.java | 218 +++++++-
 .../apache/drill/exec/proto/UserBitShared.java  | 531 ++++++++++++++++++-
 .../org/apache/drill/exec/proto/UserProtos.java | 258 +++++++--
 protocol/src/main/protobuf/BitControl.proto     |   1 +
 protocol/src/main/protobuf/User.proto           |   2 +
 protocol/src/main/protobuf/UserBitShared.proto  |   6 +-
 sqlparser/pom.xml                               |   8 +
 .../drill/jdbc/DrillColumnMetaDataList.java     |  14 +-
 .../apache/drill/jdbc/test/TestJdbcQuery.java   |  28 +-
 182 files changed, 5178 insertions(+), 2401 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
----------------------------------------------------------------------
diff --git a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
index 78c356a..700bd72 100644
--- a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
+++ b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprLexer.g
@@ -54,7 +54,7 @@ INTERVAL : 'interval' | 'INTERVAL';
 INTERVALYEAR : 'intervalyear' | 'INTERVALYEAR';
 INTERVALDAY : 'intervalday' | 'INTERVALDAY';
 
-
+Period : '.';
 Or       : '||' | 'or' | 'OR' | 'Or';
 And      : '&&' | 'and' | 'AND' ;
 Equals   : '==' | '=';
@@ -100,15 +100,21 @@ Number
 //  ;
   
 Identifier
-  :  ('a'..'z' | 'A'..'Z' | '_' | '$') ('a'..'z' | 'A'..'Z' | '_' | '$' | Digit)* ('.' ('a'..'z' | 'A'..'Z' | '_' | '$' ) ('a'..'z' | 'A'..'Z' | '_' | '$' | Digit)*)*
+  : ('a'..'z' | 'A'..'Z' | '_' | '$') ('a'..'z' | 'A'..'Z' | '_' | '$' | Digit)*
+  ;
+
+QuotedIdentifier
+@after {
+  setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
+}
+  :  '`'  (~('`' | '\\')  | '\\' ('\\' | '`'))* '`' 
   ;
 
 String
 @after {
   setText(getText().substring(1, getText().length()-1).replaceAll("\\\\(.)", "$1"));
 }
-  :  '"'  (~('"' | '\\')  | '\\' ('\\' | '"'))* '"' 
-  |  '\'' (~('\'' | '\\') | '\\' ('\\' | '\''))* '\''
+  :  '\'' (~('\'' | '\\') | '\\' ('\\' | '\''))* '\''
   ;
 
 Comment
@@ -128,3 +134,14 @@ fragment Int
 fragment Digit 
   :  '0'..'9'
   ;
+
+FallThrough
+	@after{
+	  throw new RuntimeException(java.lang.String.format(
+	      "Encountered an illegal char on line \%d, column \%d: '\%s'", 
+	      getLine(), getCharPositionInLine(), getText()
+	    )
+	  );
+	}
+  :
+  ;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
----------------------------------------------------------------------
diff --git a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
index 2e2f3d3..e4eb149 100644
--- a/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
+++ b/common/src/main/antlr3/org/apache/drill/common/expression/parser/ExprParser.g
@@ -35,6 +35,8 @@ package org.apache.drill.common.expression.parser;
 import org.antlr.runtime.BitSet;
 import java.util.*;
 import org.apache.drill.common.expression.*;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.common.expression.PathSegment.ArraySegment;
 import org.apache.drill.common.types.*;
 import org.apache.drill.common.types.TypeProtos.*;
 import org.apache.drill.common.exceptions.ExpressionParsingException;
@@ -267,12 +269,28 @@ atom returns [LogicalExpression e]
   |  lookup {$e = $lookup.e; }
   ;
 
+pathSegment returns [NameSegment seg]
+  : s1=nameSegment {$seg = $s1.seg;}
+  ;
+
+nameSegment returns [NameSegment seg]
+  : QuotedIdentifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($QuotedIdentifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+  | Identifier ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new NameSegment($Identifier.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+  ;
+  
+arraySegment returns [PathSegment seg]
+  :  OBracket Number CBracket ( (Period s1=pathSegment) | s2=arraySegment)? {$seg = new ArraySegment($Number.text, ($s1.seg == null ? $s2.seg : $s1.seg) ); }
+  ;
+
 
 lookup returns [LogicalExpression e]
   :  functionCall {$e = $functionCall.e ;}
   | castCall {$e = $castCall.e; }
-  | Identifier {$e = new SchemaPath($Identifier.text, pos($Identifier) ); }
+  | pathSegment {$e = new SchemaPath($pathSegment.seg, pos($pathSegment.start) ); }
   | String {$e = new ValueExpressions.QuotedString($String.text, pos($String) ); }
   | OParen expression CParen  {$e = $expression.e; }
   | SingleQuote Identifier SingleQuote {$e = new SchemaPath($Identifier.text, pos($Identifier) ); }
   ;
+  
+  
+  

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
index 6fc048d..7fb0e39 100644
--- a/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
+++ b/common/src/main/java/org/apache/drill/common/config/DrillConfig.java
@@ -27,6 +27,7 @@ import java.util.concurrent.CopyOnWriteArrayList;
 
 import org.apache.drill.common.exceptions.DrillConfigurationException;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.FormatPluginConfigBase;
 import org.apache.drill.common.logical.StoragePluginConfigBase;
 import org.apache.drill.common.logical.data.LogicalOperatorBase;
@@ -48,11 +49,11 @@ public final class DrillConfig extends NestedConfig{
   private final ObjectMapper mapper;
   private final ImmutableList<String> startupArguments;
   @SuppressWarnings("restriction")  private static final long MAX_DIRECT_MEMORY = sun.misc.VM.maxDirectMemory();
-  
+
   @SuppressWarnings("unchecked")
   private volatile List<Queue<Object>> sinkQueues = new CopyOnWriteArrayList<Queue<Object>>(new Queue[1]);
 
-  
+
   @SuppressWarnings("restriction")
   @VisibleForTesting
   public DrillConfig(Config config) {
@@ -60,9 +61,10 @@ public final class DrillConfig extends NestedConfig{
 
     mapper = new ObjectMapper();
     SimpleModule deserModule = new SimpleModule("LogicalExpressionDeserializationModule")
-      .addDeserializer(LogicalExpression.class, new LogicalExpression.De(this));
-    
-    
+      .addDeserializer(LogicalExpression.class, new LogicalExpression.De(this))
+      .addDeserializer(SchemaPath.class, new SchemaPath.De(this));
+
+
     mapper.registerModule(deserModule);
     mapper.enable(SerializationFeature.INDENT_OUTPUT);
     mapper.configure(Feature.ALLOW_UNQUOTED_FIELD_NAMES, true);
@@ -71,10 +73,10 @@ public final class DrillConfig extends NestedConfig{
     mapper.registerSubtypes(LogicalOperatorBase.getSubTypes(this));
     mapper.registerSubtypes(StoragePluginConfigBase.getSubTypes(this));
     mapper.registerSubtypes(FormatPluginConfigBase.getSubTypes(this));
-    
+
     RuntimeMXBean bean = ManagementFactory.getRuntimeMXBean();
     this.startupArguments = ImmutableList.copyOf(bean.getInputArguments());
-    
+
   };
 
 
@@ -82,15 +84,15 @@ public final class DrillConfig extends NestedConfig{
   public List<String> getStartupArguments(){
     return startupArguments;
   }
-  
+
   /**
-   * Create a DrillConfig object using the default config file name 
+   * Create a DrillConfig object using the default config file name
    * @return The new DrillConfig object.
    */
   public static DrillConfig create() {
     return create(null);
   }
-  
+
   /**
    * <p>
    * DrillConfig loads up Drill configuration information. It does this utilizing a combination of classpath scanning
@@ -106,15 +108,15 @@ public final class DrillConfig extends NestedConfig{
    * <li>Check a single copy of "drill-default.conf". If multiple copies are on the classpath, behavior is
    * indeterminate.</li>
    * </ul>
-   * 
+   *
    * </p>
    *  @param overrideFileName The name of the file to use for override purposes.
    *  @return A merged Config object.
    */
   public static DrillConfig create(String overrideFileName) {
-    
+
     overrideFileName = overrideFileName == null ? CommonConstants.CONFIG_OVERRIDE : overrideFileName;
-    
+
     // first we load defaults.
     Config fallback = ConfigFactory.load(CommonConstants.CONFIG_DEFAULT);
     Collection<URL> urls = PathScanner.getConfigURLs();
@@ -126,7 +128,7 @@ public final class DrillConfig extends NestedConfig{
     Config c = ConfigFactory.load(overrideFileName).withFallback(fallback).resolve();
     return new DrillConfig(c);
   }
-  
+
   public <T> Class<T> getClassAt(String location, Class<T> clazz) throws DrillConfigurationException{
     String className = this.getString(location);
     if(className == null) throw new DrillConfigurationException(String.format("No class defined at location '%s'.  Expected a definition of the class []", location, clazz.getCanonicalName()));
@@ -142,9 +144,9 @@ public final class DrillConfig extends NestedConfig{
       if(ex instanceof DrillConfigurationException) throw (DrillConfigurationException) ex;
       throw new DrillConfigurationException(String.format("Failure while initializing class [%s] described at configuration value '%s'.", className, location), ex);
     }
-    
+
   }
-  
+
   public <T> T getInstanceOf(String location, Class<T> clazz) throws DrillConfigurationException{
     Class<T> c = getClassAt(location, clazz);
     try{
@@ -154,18 +156,18 @@ public final class DrillConfig extends NestedConfig{
       throw new DrillConfigurationException(String.format("Failure while instantiating class [%s] located at '%s.", clazz.getCanonicalName(), location), ex);
     }
   }
-  
 
-  
+
+
   public void setSinkQueues(int number, Queue<Object> queue){
     sinkQueues.set(number, queue);
   }
-  
+
   public Queue<Object> getQueue(int number){
     if(sinkQueues.size() <= number || number < 0 || sinkQueues == null) throw new IllegalArgumentException(String.format("Queue %d is not available.", number));
     return sinkQueues.get(number);
   }
-  
+
   public ObjectMapper getMapper(){
     return mapper;
   }
@@ -174,13 +176,13 @@ public final class DrillConfig extends NestedConfig{
   public String toString(){
     return this.root().render();
   }
-  
+
   public static void main(String[] args)  throws Exception{
     //"-XX:MaxDirectMemorySize"
     DrillConfig config = DrillConfig.create();
-    
+
   }
-  
+
   public static long getMaxDirectMemory(){
     return MAX_DIRECT_MEMORY;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
index 787e4af..89fa437 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ExpressionStringBuilder.java
@@ -44,6 +44,19 @@ import com.google.common.collect.ImmutableList;
 public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBuilder, RuntimeException>{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExpressionStringBuilder.class);
 
+  static final ExpressionStringBuilder INSTANCE = new ExpressionStringBuilder();
+
+  public static String toString(LogicalExpression expr){
+    StringBuilder sb = new StringBuilder();
+    expr.accept(INSTANCE, sb);
+    return sb.toString();
+  }
+
+  public static void toString(LogicalExpression expr, StringBuilder sb){
+    expr.accept(INSTANCE, sb);
+  }
+
+
   @Override
   public Void visitFunctionCall(FunctionCall call, StringBuilder sb) throws RuntimeException {
     ImmutableList<LogicalExpression> args = call.args;
@@ -85,7 +98,26 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
 
   @Override
   public Void visitSchemaPath(SchemaPath path, StringBuilder sb) throws RuntimeException {
-    sb.append(path.getPath());
+    PathSegment seg = path.getRootSegment();
+    if(seg.isArray()) throw new IllegalStateException("Drill doesn't currently support top level arrays");
+    sb.append('`');
+    sb.append(seg.getNameSegment().getPath());
+    sb.append('`');
+
+    while( (seg = seg.getChild()) != null){
+      if(seg.isNamed()){
+        sb.append('.');
+        sb.append('`');
+        sb.append(seg.getNameSegment().getPath());
+        sb.append('`');
+      }else{
+        sb.append('[');
+        sb.append(seg.getArraySegment().getIndex());
+        sb.append(']');
+      }
+    }
+
+
     return null;
   }
 
@@ -121,33 +153,33 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
     TIMESTAMP_FORMAT = new DateTimeFormatterBuilder().append(dateFormatter).appendOptional(optionalTime).appendOptional(optionalSec).toFormatter();
 
   }
-  
+
   @Override
   public Void visitTimeStampConstant(TimeStampExpression lExpr, StringBuilder sb) throws RuntimeException {
-    sb.append("cast( \"");
+    sb.append("cast( '");
     try {
       TIMESTAMP_FORMAT.printTo(sb, lExpr.getTimeStamp());
     } catch (IOException e) {
       throw new RuntimeException(e);
     }
     sb.append(lExpr.getTimeStamp());
-    sb.append(" \" as TIMESTAMP)");
+    sb.append(" ' as TIMESTAMP)");
     return null;
   }
 
   @Override
   public Void visitIntervalYearConstant(IntervalYearExpression lExpr, StringBuilder sb) throws RuntimeException {
-    sb.append("cast( \"");
+    sb.append("cast( '");
     sb.append(Period.years(lExpr.getIntervalYear()).toString());
-    sb.append("\" as INTERVALYEAR)");
+    sb.append("' as INTERVALYEAR)");
     return null;
   }
 
   @Override
   public Void visitIntervalDayConstant(IntervalDayExpression lExpr, StringBuilder sb) throws RuntimeException {
-    sb.append("cast( \"");
+    sb.append("cast( '");
     sb.append(Period.days(lExpr.getIntervalDay()).plusMillis(lExpr.getIntervalMillis()).toString());
-    sb.append("\" as INTERVALDAY)");
+    sb.append("' as INTERVALDAY)");
     return null;
   }
 
@@ -165,21 +197,21 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
 
   @Override
   public Void visitQuotedStringConstant(QuotedString e, StringBuilder sb) throws RuntimeException {
-    sb.append("\"");
+    sb.append("'");
     sb.append(e.value);
-    sb.append("\"");
+    sb.append("'");
     return null;
   }
 
   @Override
   public Void visitCastExpression(CastExpression e, StringBuilder sb) throws RuntimeException {
     MajorType mt = e.getMajorType();
-    
+
     sb.append("cast( (");
     e.getInput().accept(this, sb);
     sb.append(" ) as ");
     sb.append(mt.getMinorType().name());
-    
+
     switch(mt.getMinorType()){
     case FLOAT4:
     case FLOAT8:
@@ -228,8 +260,8 @@ public class ExpressionStringBuilder extends AbstractExprVisitor<Void, StringBui
     sb.append(intExpr.getInt());
     return null;
   }
-  
-  
-  
-  
+
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FieldReference.java b/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
index 358a865..bcc6b52 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FieldReference.java
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.drill.common.expression.FieldReference.De;
 import org.apache.drill.common.expression.FieldReference.Se;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 
 import com.fasterxml.jackson.core.JsonGenerationException;
@@ -41,18 +42,32 @@ public class FieldReference extends SchemaPath {
 
   public FieldReference(SchemaPath sp){
     super(sp);
+    checkData();
   }
-  
+
+  private void checkData(){
+    if(getRootSegment().getChild() != null) throw new UnsupportedOperationException("Field references must be singular names.");
+
+  }
+
+
+  private void checkSimpleString(CharSequence value){
+    if(value.toString().contains(".")) throw new UnsupportedOperationException("Field references must be singular names.");
+  }
+
   public FieldReference(CharSequence value){
-    super(value, ExpressionPosition.UNKNOWN);
+    this(value, ExpressionPosition.UNKNOWN);
+    checkSimpleString(value);
   }
-  
+
   public FieldReference(CharSequence value, ExpressionPosition pos) {
-    super(value, pos);
+    super(new NameSegment(value), pos);
+    checkData();
+    checkSimpleString(value);
   }
 
   public FieldReference(String value, ExpressionPosition pos, MajorType dataType) {
-    super(value, pos);
+    this(value, pos);
     this.overrideType = dataType;
   }
 
@@ -74,7 +89,9 @@ public class FieldReference extends SchemaPath {
     @Override
     public FieldReference deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
         JsonProcessingException {
-      return new FieldReference(this._parseString(jp, ctxt), ExpressionPosition.UNKNOWN);
+      String ref = this._parseString(jp, ctxt);
+      ref = ref.replace("`", "");
+      return new FieldReference(ref, ExpressionPosition.UNKNOWN);
     }
 
   }
@@ -88,7 +105,7 @@ public class FieldReference extends SchemaPath {
     @Override
     public void serialize(FieldReference value, JsonGenerator jgen, SerializerProvider provider) throws IOException,
         JsonGenerationException {
-      jgen.writeString(value.getPath().toString());
+      jgen.writeString('`' + value.getRootSegment().getNameSegment().getPath() + '`');
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java b/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
index 4df2c8e..cceaef0 100644
--- a/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
+++ b/common/src/main/java/org/apache/drill/common/expression/LogicalExpression.java
@@ -103,4 +103,6 @@ public interface LogicalExpression extends Iterable<LogicalExpression>{
 
   }
 
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/PathSegment.java b/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
index ac1aeac..448e50f 100644
--- a/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
+++ b/common/src/main/java/org/apache/drill/common/expression/PathSegment.java
@@ -17,38 +17,42 @@
  */
 package org.apache.drill.common.expression;
 
-import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
 
 
 
 public abstract class PathSegment{
 
-  protected final ValueExpressions.CollisionBehavior collision;
   protected PathSegment child;
 
-  protected PathSegment(CollisionBehavior collision){
-    this.collision = collision;
-  }
-  
+  public abstract PathSegment cloneWithNewChild(PathSegment segment);
   public abstract PathSegment clone();
-  
+
   public static class ArraySegment extends PathSegment{
     private final int index;
-    
-    public ArraySegment(int index, ValueExpressions.CollisionBehavior collision){
-      super(collision);
+
+    public ArraySegment(String numberAsText, PathSegment child){
+      this.child = child;
+      this.index = Integer.parseInt(numberAsText);
+    }
+
+    public ArraySegment(int index, PathSegment child){
+      this.child = child;
+      this.index = index;
+    }
+
+    public ArraySegment(int index){
       if(index < 0 ) throw new IllegalArgumentException();
       this.index = index;
     }
-    
+
     public int getIndex(){
       return index;
     }
-    
+
     public boolean isArray(){
       return true;
     }
-    
+
     public boolean isNamed(){
       return false;
     }
@@ -60,37 +64,49 @@ public abstract class PathSegment{
 
     @Override
     public String toString() {
-      return "ArraySegment [index=" + index + ", getCollisionBehavior()=" + getCollisionBehavior() + ", getChild()="
-          + getChild() + "]";
+      return "ArraySegment [index=" + index + ", getChild()=" + getChild() + "]";
     }
 
     @Override
     public PathSegment clone() {
-      PathSegment seg = new ArraySegment(index, collision);
+      PathSegment seg = new ArraySegment(index);
       if(child != null) seg.setChild(child.clone());
       return seg;
     }
-    
+
+    public ArraySegment cloneWithNewChild(PathSegment newChild){
+      ArraySegment seg = new ArraySegment(index);
+      if(child != null){
+        seg.setChild(child.cloneWithNewChild(newChild));
+      }else{
+        seg.setChild(newChild);
+      }
+      return seg;
+    }
   }
-  
-  
-  
+
+
+
   public static class NameSegment extends PathSegment{
     private final String path;
-    
-    public NameSegment(CharSequence n, ValueExpressions.CollisionBehavior collision){
-      super(collision);
+
+    public NameSegment(CharSequence n, PathSegment child){
+      this.child = child;
+      this.path = n.toString();
+    }
+
+    public NameSegment(CharSequence n){
       this.path = n.toString();
     }
-    
+
     public String getPath(){
-      return path;        
+      return path;
     }
-    
+
     public boolean isArray(){
       return false;
     }
-    
+
     public boolean isNamed(){
       return true;
     }
@@ -102,15 +118,14 @@ public abstract class PathSegment{
 
     @Override
     public String toString() {
-      return "NameSegment [path=" + path + ", getCollisionBehavior()=" + getCollisionBehavior() + ", getChild()="
-          + getChild() + "]";
+      return "NameSegment [path=" + path + ", getChild()=" + getChild() + "]";
     }
 
     @Override
     public int hashCode() {
       final int prime = 31;
       int result = 1;
-      result = prime * result + ((path == null) ? 0 : path.hashCode());
+      result = prime * result + ((path == null) ? 0 : path.toLowerCase().hashCode());
       return result;
     }
 
@@ -132,16 +147,27 @@ public abstract class PathSegment{
     }
 
     @Override
-    public PathSegment clone() {
-      PathSegment s = new NameSegment(this.path, this.collision);
+    public NameSegment clone() {
+      NameSegment s = new NameSegment(this.path);
       if(child != null) s.setChild(child.clone());
       return s;
     }
-    
-    
-    
+
+
+    @Override
+    public NameSegment cloneWithNewChild(PathSegment newChild) {
+      NameSegment s = new NameSegment(this.path);
+      if(child != null){
+        s.setChild(child.cloneWithNewChild(newChild));
+      }else{
+        s.setChild(newChild);
+      }
+      return s;
+    }
+
+
   }
-  
+
   public NameSegment getNameSegment(){
     throw new UnsupportedOperationException();
   }
@@ -150,12 +176,8 @@ public abstract class PathSegment{
   }
   public abstract boolean isArray();
   public abstract boolean isNamed();
-  
-  
-  public ValueExpressions.CollisionBehavior getCollisionBehavior(){
-    return collision;
-  }
-  
+
+
   public boolean isLastPath(){
     return child == null;
   }
@@ -168,6 +190,6 @@ public abstract class PathSegment{
     this.child = child;
   }
 
-  
-  
+
+
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java b/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
index 76838dc..a3f93a3 100644
--- a/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
+++ b/common/src/main/java/org/apache/drill/common/expression/SchemaPath.java
@@ -17,98 +17,79 @@
  */
 package org.apache.drill.common.expression;
 
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
 
-import org.apache.drill.common.expression.ValueExpressions.CollisionBehavior;
+import org.antlr.runtime.ANTLRStringStream;
+import org.antlr.runtime.CommonTokenStream;
+import org.antlr.runtime.RecognitionException;
+import org.apache.drill.common.config.DrillConfig;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
+import org.apache.drill.common.expression.parser.ExprLexer;
+import org.apache.drill.common.expression.parser.ExprParser;
+import org.apache.drill.common.expression.parser.ExprParser.parse_return;
 import org.apache.drill.common.expression.visitors.ExprVisitor;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.Types;
 
-import com.google.common.base.Joiner;
+import com.fasterxml.jackson.core.JsonGenerationException;
+import com.fasterxml.jackson.core.JsonGenerator;
+import com.fasterxml.jackson.core.JsonParser;
+import com.fasterxml.jackson.core.JsonProcessingException;
+import com.fasterxml.jackson.databind.DeserializationContext;
+import com.fasterxml.jackson.databind.SerializerProvider;
+import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
+import com.fasterxml.jackson.databind.ser.std.StdSerializer;
 import com.google.common.collect.Iterators;
-import com.google.protobuf.DescriptorProtos.UninterpretedOption.NamePart;
 
 public class SchemaPath extends LogicalExpressionBase {
 
-  // reads well in RegexBuddy
-  private static final String ENTIRE_REGEX = "^\n" + "(?:                # first match required\n"
-      + "\\[\\d+\\]             # array index only\n" + "|\n" + "'?\n" + "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n"
-      + "'?\n" + "(?:\\[\\d+\\])?\n" + ")\n" + "[\\+\\-\\!\\]\\}]?\n" +
+  private final NameSegment rootSegment;
 
-      "# secondary matches (starts with dot)\n" + "(?:\n" + "\\.\n" + "(?:                # first match required\n"
-      + "\\[\\d+\\]             # array index only\n" + "|\n" + "'?\n" + "[^\\.\\[\\+\\-\\!\\]\\}]+  # identifier\n"
-      + "'?\n" + "(?:\\[\\d+\\])?\n" + ")\n" + "[\\+\\-\\!\\]\\}]?\n" +
 
-      ")*$";
+  public static SchemaPath getSimplePath(String name){
+    return getCompoundPath(name);
+  }
+
+  public static SchemaPath getCompoundPath(String... strings){
+    List<String> paths = Arrays.asList(strings);
+    Collections.reverse(paths);
+    NameSegment s = null;
+    for(String p : paths){
+      s = new NameSegment(p);
+    }
+    return new SchemaPath(s);
+  }
 
-  // reads well in RegexBuddy
-  private static final String SEGMENT_REGEX = "(?:\n" + "\\[(\\d+)\\]\n" + "|\n" + "'?\n"
-      + "([^\\.\\[\\+\\-\\!\\]\\}]+)  # identifier\n" + "'?\n" + ")\n"
-      + "([\\+\\-\\!\\]\\}]?)         # collision type";
-  private static final int GROUP_INDEX = 1;
-  private static final int GROUP_PATH_SEGMENT = 2;
-  private static final int GROUP_COLLISION = 3;
 
-  private final static Pattern SEGMENT_PATTERN = Pattern.compile(SEGMENT_REGEX, Pattern.COMMENTS);
-  private final static Pattern ENTIRE_PATTERN = Pattern.compile(ENTIRE_REGEX, Pattern.COMMENTS);
 
-  private final CharSequence originalPath;
-  private final PathSegment rootSegment;
+  /**
+   *
+   * @param simpleName
+   */
+  @Deprecated
+  public SchemaPath(String simpleName, ExpressionPosition pos){
+    super(pos);
+    this.rootSegment = new NameSegment(simpleName);
+    if(simpleName.contains(".")) throw new IllegalStateException("This is deprecated and only supports simpe paths.");
+  }
 
   public SchemaPath(SchemaPath path){
     super(path.getPosition());
-    this.originalPath = path.originalPath;
     this.rootSegment = path.rootSegment;
   }
 
-  public SchemaPath(String str){
-    this(str, ExpressionPosition.UNKNOWN);
+  public SchemaPath(NameSegment rootSegment){
+    super(ExpressionPosition.UNKNOWN);
+    this.rootSegment = rootSegment;
   }
-  
-  public SchemaPath(CharSequence str, ExpressionPosition pos) {
-    super(pos);
 
-    if (!ENTIRE_PATTERN.matcher(str).matches())
-      throw new IllegalArgumentException("Identifier doesn't match expected pattern.");
-    this.originalPath = str;
-    Matcher m = SEGMENT_PATTERN.matcher(str);
-    PathSegment r = null;
-    PathSegment previous = null;
-    PathSegment current;
-    while (m.find()) {
-      CollisionBehavior col = (m.start(GROUP_COLLISION) != -1) ? CollisionBehavior.find(m.group(GROUP_COLLISION))
-          : CollisionBehavior.DEFAULT;
-
-      if (m.start(GROUP_INDEX) != -1) {
-        String d = m.group(GROUP_INDEX);
-        current = new PathSegment.ArraySegment(Integer.parseInt(d), col);
-      } else {
-        String i = m.group(GROUP_PATH_SEGMENT);
-        current = new PathSegment.NameSegment(i, col);
-      }
-      if (previous == null) {
-        r = current;
-      } else {
-        previous.setChild(current);
-      }
-      previous = current;
-    }
-
-    rootSegment = r;
-
-  }
-  
-  private SchemaPath(SchemaPath parent, String[] childPaths){
-    super(ExpressionPosition.UNKNOWN);
-    SchemaPath p = new SchemaPath(Joiner.on('.').join(childPaths), ExpressionPosition.UNKNOWN);
-    this.originalPath = parent.originalPath + "." + p.originalPath;
-    PathSegment seg = parent.getRootSegment().clone();
-    this.rootSegment = seg;
-    while(!seg.isLastPath()) seg = seg.getChild();
-    seg.setChild(p.getRootSegment());
+  public SchemaPath(NameSegment rootSegment, ExpressionPosition pos){
+    super(pos);
+    this.rootSegment = rootSegment;
   }
 
   @Override
@@ -116,16 +97,13 @@ public class SchemaPath extends LogicalExpressionBase {
     return visitor.visitSchemaPath(this, value);
   }
 
-  public SchemaPath getChild(String... childPaths){
-    return new SchemaPath(this, childPaths);
-  }
-  
-  public PathSegment getRootSegment() {
-    return rootSegment;
+  public SchemaPath getChild(String childPath){
+    rootSegment.cloneWithNewChild(new NameSegment(childPath));
+    return new SchemaPath(rootSegment);
   }
 
-  public CharSequence getPath() {
-    return originalPath;
+  public NameSegment getRootSegment() {
+    return rootSegment;
   }
 
   @Override
@@ -147,7 +125,7 @@ public class SchemaPath extends LogicalExpressionBase {
       return true;
     if (obj == null)
       return false;
-    if (getClass() != obj.getClass())
+    if ( !(obj instanceof SchemaPath))
       return false;
     SchemaPath other = (SchemaPath) obj;
     if (rootSegment == null) {
@@ -163,9 +141,54 @@ public class SchemaPath extends LogicalExpressionBase {
     return Iterators.emptyIterator();
   }
 
+
   @Override
   public String toString() {
-    return "SchemaPath [rootSegment=" + rootSegment + "]";
+    String expr = ExpressionStringBuilder.toString(this);
+    return "SchemaPath ["+ expr + "]";
+  }
+
+  public String toExpr(){
+    return ExpressionStringBuilder.toString(this);
+  }
+
+
+  public static class De extends StdDeserializer<SchemaPath> {
+    DrillConfig config;
+
+    public De(DrillConfig config) {
+      super(LogicalExpression.class);
+      this.config = config;
+    }
+
+    @Override
+    public SchemaPath deserialize(JsonParser jp, DeserializationContext ctxt) throws IOException,
+        JsonProcessingException {
+      String expr = jp.getText();
+
+      if (expr == null || expr.isEmpty())
+        return null;
+      try {
+        // logger.debug("Parsing expression string '{}'", expr);
+        ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
+        CommonTokenStream tokens = new CommonTokenStream(lexer);
+        ExprParser parser = new ExprParser(tokens);
+
+        //TODO: move functionregistry and error collector to injectables.
+        //ctxt.findInjectableValue(valueId, forProperty, beanInstance)
+        parse_return ret = parser.parse();
+
+        // ret.e.resolveAndValidate(expr, errorCollector);
+        if(ret.e instanceof SchemaPath){
+          return (SchemaPath) ret.e;
+        }else{
+          throw new IllegalStateException("Schema path is not a valid format.");
+        }
+      } catch (RecognitionException e) {
+        throw new RuntimeException(e);
+      }
+    }
+
   }
 
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
index 4b83e7d..1d217f2 100644
--- a/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
+++ b/common/src/main/java/org/apache/drill/common/expression/ValueExpressions.java
@@ -33,22 +33,22 @@ public class ValueExpressions {
   public static LogicalExpression getBigInt(long l){
     return new LongExpression(l);
   }
-  
+
   public static LogicalExpression getInt(int i){
     return new IntExpression(i, ExpressionPosition.UNKNOWN);
   }
-  
+
   public static LogicalExpression getFloat8(double d){
     return new DoubleExpression(d, ExpressionPosition.UNKNOWN);
   }
   public static LogicalExpression getFloat4(float f){
     return new DoubleExpression(f, ExpressionPosition.UNKNOWN);
   }
-  
+
   public static LogicalExpression getBit(boolean b){
     return new BooleanExpression(Boolean.toString(b), ExpressionPosition.UNKNOWN);
   }
-  
+
   public static LogicalExpression getChar(String s){
     return new QuotedString(s, ExpressionPosition.UNKNOWN);
   }
@@ -72,7 +72,7 @@ public class ValueExpressions {
       return new IntervalDayExpression(intervalInMillis);
   }
 
-  
+
 
   public static LogicalExpression getNumericExpression(String s, ExpressionPosition ep) {
     try {
@@ -109,7 +109,7 @@ public class ValueExpressions {
     }
 
     protected abstract V parseValue(String s);
-    
+
     @Override
     public Iterator<LogicalExpression> iterator() {
       return Iterators.emptyIterator();
@@ -119,13 +119,13 @@ public class ValueExpressions {
   }
 
   public static class BooleanExpression extends ValueExpression<Boolean> {
-    
-    
+
+
     public BooleanExpression(String value, ExpressionPosition pos) {
       super(value, pos);
     }
 
-    
+
     @Override
     protected Boolean parseValue(String s) {
       return Boolean.parseBoolean(s);
@@ -250,7 +250,7 @@ public class ValueExpressions {
     public LongExpression(long l) {
       this(l, ExpressionPosition.UNKNOWN);
     }
-    
+
       public LongExpression(long l, ExpressionPosition pos) {
       super(pos);
       this.l = l;
@@ -269,7 +269,7 @@ public class ValueExpressions {
     public <T, V, E extends Exception> T accept(ExprVisitor<T, V, E> visitor, V value) throws E {
       return visitor.visitLongConstant(this, value);
     }
-    
+
     @Override
     public Iterator<LogicalExpression> iterator() {
       return Iterators.emptyIterator();
@@ -427,10 +427,10 @@ public class ValueExpressions {
 
     private static final MajorType INTERVALDAY_CONSTANT = Types.required(MinorType.INTERVALDAY);
     private static final long MILLIS_IN_DAY = 1000 * 60 * 60 * 24;
-    
+
     private int days;
     private int millis;
-    
+
     public IntervalDayExpression(long intervalInMillis) {
       this((int) (intervalInMillis / MILLIS_IN_DAY), (int) (intervalInMillis % MILLIS_IN_DAY), ExpressionPosition.UNKNOWN);
     }
@@ -490,34 +490,4 @@ public class ValueExpressions {
     }
   }
 
-  public static enum CollisionBehavior {
-    SKIP("-"), // keep the old value.
-    FAIL("!"), // give up on the record
-    REPLACE("+"), // replace the old value with the new value.
-    ARRAYIFY("]"), // replace the current position with an array. Then place the
-                   // old and new value in the array.
-    OBJECTIFY("}"), // replace the current position with a map. Give the two
-                    // values names of 'old' and 'new'.
-    MERGE_OVERRIDE("%"); // do your best to do a deep merge of the old and new
-                         // values.
-
-    private String identifier;
-
-    private CollisionBehavior(String identifier) {
-      this.identifier = identifier;
-    }
-
-    public static final CollisionBehavior DEFAULT = FAIL;
-
-    public static final CollisionBehavior find(String c) {
-      if (c == null || c.isEmpty())
-        return DEFAULT;
-
-      for (CollisionBehavior b : values()) {
-        if (b.identifier.equals(c))
-          return b;
-      }
-      return DEFAULT;
-    }
-  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/logical/data/Project.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/logical/data/Project.java b/common/src/main/java/org/apache/drill/common/logical/data/Project.java
index 645dc32..ab2ef35 100644
--- a/common/src/main/java/org/apache/drill/common/logical/data/Project.java
+++ b/common/src/main/java/org/apache/drill/common/logical/data/Project.java
@@ -43,17 +43,17 @@ public class Project extends SingleInputOperator {
     if (selections == null || selections.length == 0)
       throw new ExpressionParsingException(
           "Project did not provide any projection selections.  At least one projection must be provided.");
-    for (int i = 0; i < selections.length; i++) {
-      PathSegment segment = selections[i].getRef().getRootSegment();
-      CharSequence path = segment.getNameSegment().getPath();
-      if (!segment.isNamed() || !path.equals("output"))
-        throw new ExpressionParsingException(
-            String
-                .format(
-                    "Outputs for projections always have to start with named path of output. First segment was named '%s' or was named [%s]",
-                    path, segment.isNamed()));
-
-    }
+//    for (int i = 0; i < selections.length; i++) {
+//      PathSegment segment = selections[i].getRef().getRootSegment();
+//      CharSequence path = segment.getNameSegment().getPath();
+//      if (!segment.isNamed() || !path.equals("output"))
+//        throw new ExpressionParsingException(
+//            String
+//                .format(
+//                    "Outputs for projections always have to start with named path of output. First segment was named '%s' or was named [%s]",
+//                    path, segment.isNamed()));
+//
+//    }
   }
 
   @JsonProperty("projections")

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/main/java/org/apache/drill/common/util/TestTools.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/util/TestTools.java b/common/src/main/java/org/apache/drill/common/util/TestTools.java
index 0d3cc3e..6e0c241 100644
--- a/common/src/main/java/org/apache/drill/common/util/TestTools.java
+++ b/common/src/main/java/org/apache/drill/common/util/TestTools.java
@@ -21,23 +21,57 @@ import java.nio.file.Paths;
 
 import org.junit.rules.TestName;
 import org.junit.rules.TestRule;
+import org.junit.rules.TestWatcher;
 import org.junit.rules.Timeout;
+import org.junit.runner.Description;
+import org.slf4j.Logger;
 
 public class TestTools {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTools.class);
-  
-  static final boolean IS_DEBUG = java.lang.management.ManagementFactory.getRuntimeMXBean().getInputArguments().toString().indexOf("-agentlib:jdwp") > 0;
+
+  static final boolean IS_DEBUG = java.lang.management.ManagementFactory.getRuntimeMXBean().getInputArguments()
+      .toString().indexOf("-agentlib:jdwp") > 0;
   static final String WORKING_PATH = Paths.get("").toAbsolutePath().toString();
-  
-  public static TestRule getTimeoutRule(){
+
+  public static TestRule getTimeoutRule() {
     return getTimeoutRule(10000);
   }
 
-  public static TestRule getTimeoutRule(int timeout){
+  public static TestRule getTimeoutRule(int timeout) {
     return IS_DEBUG ? new TestName() : new Timeout(timeout);
   }
 
-  public static String getWorkingPath(){
+  public static String getWorkingPath() {
     return WORKING_PATH;
   }
+
+  public static TestLogReporter getTestLogReporter(final Logger logger) {
+    return new TestLogReporter(logger);
+  }
+
+  public static class TestLogReporter extends TestWatcher{
+    private int failureCount = 0;
+
+    final Logger logger;
+
+    public TestLogReporter(Logger logger) {
+      super();
+      this.logger = logger;
+    }
+
+    @Override
+    protected void failed(Throwable e, Description description) {
+      logger.error("Test Failed: " + description.getDisplayName(), e);
+      failureCount++;
+    }
+
+    @Override
+    public void succeeded(Description description) {
+      logger.info("Test Succeeded: " + description.getDisplayName());
+    }
+
+    public void sleepIfFailure() throws InterruptedException{
+      if(failureCount > 0) Thread.sleep(2000);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/common/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
----------------------------------------------------------------------
diff --git a/common/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java b/common/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
index f49b042..de090df 100644
--- a/common/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
+++ b/common/src/test/java/org/apache/drill/common/expression/parser/TreeTest.java
@@ -17,46 +17,98 @@
  */
 package org.apache.drill.common.expression.parser;
 
-import java.io.File;
+import java.io.IOException;
 
 import org.antlr.runtime.ANTLRStringStream;
 import org.antlr.runtime.CommonTokenStream;
-import org.antlr.runtime.tree.CommonTree;
-import org.antlr.runtime.tree.DOTTreeGenerator;
-import org.antlr.stringtemplate.StringTemplate;
+import org.antlr.runtime.RecognitionException;
+import org.apache.drill.common.expression.ExpressionStringBuilder;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.parser.ExprParser.parse_return;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.SerializationFeature;
-import com.google.common.base.Charsets;
-import com.google.common.io.Files;
+import org.junit.Assert;
+import org.junit.Test;
 
 public class TreeTest {
-  public static void main(String[] args) throws Exception {
 
-    ExprLexer lexer = new ExprLexer(
-        new ANTLRStringStream(
-            "if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end"));
-    // ExprLexer lexer = new ExprLexer(new
-    // ANTLRStringStream("if ('blue.red') then 'orange' else if (false) then 1 else 0 end"));
-    // ExprLexer lexer = new ExprLexer(new ANTLRStringStream("2+2"));
 
+  @Test
+  public void testIfWithCase() throws Exception{
+    testExpressionParsing("if ($F1) then case when (_MAP.R_NAME = 'AFRICA') then 2 else 4 end else if(4==3) then 1 else if(x==3) then 7 else (if(2==1) then 6 else 4 end) end");
+  }
+
+  @Test
+  public void testAdd() throws Exception{
+    testExpressionParsing("2+2");
+  }
+
+  @Test
+  public void testIf() throws Exception{
+    testExpressionParsing("if ('blue.red') then 'orange' else if (false) then 1 else 0 end");
+  }
+
+  @Test
+  public void testQuotedIdentifier() throws Exception{
+    testExpressionParsing("`hello friend`.`goodbye`");
+  }
+
+  @Test
+  public void testSpecialQuoted() throws Exception{
+    testExpressionParsing("`*0` + `*` ");
+  }
+
+  @Test
+  public void testQuotedIdentifier2() throws Exception{
+    testExpressionParsing("`hello friend`.goodbye");
+  }
+
+  @Test
+  public void testComplexIdentifier() throws Exception{
+    testExpressionParsing("goodbye[4].`hello`");
+  }
+
+  private LogicalExpression parseExpression(String expr) throws RecognitionException, IOException{
+
+    ExprLexer lexer = new ExprLexer(new ANTLRStringStream(expr));
     CommonTokenStream tokens = new CommonTokenStream(lexer);
 
+//    tokens.fill();
+//    for(Token t : (List<Token>) tokens.getTokens()){
+//      System.out.println(t + "" + t.getType());
+//    }
+//    tokens.rewind();
+
     ExprParser parser = new ExprParser(tokens);
     parse_return ret = parser.parse();
-    LogicalExpression e = ret.e;
-    ObjectMapper mapper = new ObjectMapper();
-    mapper.enable(SerializationFeature.INDENT_OUTPUT);
-    System.out.println(mapper.writeValueAsString(e));
 
-    // print the tree
-    CommonTree tree = (CommonTree) ret.getTree();
-    DOTTreeGenerator gen = new DOTTreeGenerator();
-    StringTemplate st = gen.toDOT(tree);
+    return ret.e;
 
-    Files.write(st.toString(), new File("/Users/jnadeau/Documents/tree.dot"), Charsets.UTF_8);
+  }
+
+  private String serializeExpression(LogicalExpression expr){
 
+    ExpressionStringBuilder b = new ExpressionStringBuilder();
+    StringBuilder sb = new StringBuilder();
+    expr.accept(b, sb);
+    return sb.toString();
   }
+
+  /**
+   * Attempt to parse an expression.  Once parsed, convert it to a string and then parse it again to make sure serialization works.
+   * @param expr
+   * @throws RecognitionException
+   * @throws IOException
+   */
+  private void testExpressionParsing(String expr) throws RecognitionException, IOException{
+    System.out.println("-----" + expr + "-----");
+    LogicalExpression e = parseExpression(expr);
+
+    String newStringExpr = serializeExpression(e);
+    System.out.println(newStringExpr);
+    LogicalExpression e2 = parseExpression(newStringExpr);
+    //Assert.assertEquals(e, e2);
+
+  }
+
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java
index 0211de0..8e1e0ac 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseRecordReader.java
@@ -17,16 +17,17 @@
  */
 package org.apache.drill.exec.store.hbase;
 
-import com.google.common.base.Preconditions;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hasher;
-import com.google.common.hash.Hashing;
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.TimeUnit;
+
 import org.apache.drill.common.exceptions.DrillRuntimeException;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.Types;
@@ -48,15 +49,19 @@ import org.apache.hadoop.hbase.client.ResultScanner;
 import org.apache.hadoop.hbase.client.Scan;
 import org.apache.hadoop.hbase.util.Bytes;
 
-import java.io.IOException;
-import java.util.*;
-import java.util.concurrent.TimeUnit;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hasher;
+import com.google.common.hash.Hashing;
 
 public class HBaseRecordReader implements RecordReader {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HBaseRecordReader.class);
 
   private static final String ROW_KEY = "row_key";
   private static final int TARGET_RECORD_COUNT = 4000;
+  private static final SchemaPath ROW_KEY_PATH = SchemaPath.getSimplePath(ROW_KEY);
 
   private List<SchemaPath> columns;
   private OutputMutator outputMutator;
@@ -75,16 +80,16 @@ public class HBaseRecordReader implements RecordReader {
     this.context = context;
     if (columns != null && columns.size() != 0) {
       for (SchemaPath column : columns) {
-        if (column.getPath().toString().equalsIgnoreCase(ROW_KEY)) {
-          rowKeySchemaPath = new SchemaPath(ROW_KEY);
+        if (column.getRootSegment().getPath().toString().equalsIgnoreCase(ROW_KEY)) {
+          rowKeySchemaPath = ROW_KEY_PATH;
           continue;
         }
-        PathSegment root = column.getRootSegment();
+        NameSegment root = column.getRootSegment();
         assert root != null;
         PathSegment child = root.getChild();
-        byte[] family = root.getNameSegment().getPath().toString().getBytes();
+        byte[] family = root.getPath().toString().getBytes();
         if (child != null) {
-          Preconditions.checkArgument(child.getChild() == null, "Unsupported column name: " + column.getPath());
+          Preconditions.checkArgument(child.getChild() == null, "Unsupported column name: " + column.toString());
           byte[] qualifier = child.getNameSegment().getPath().toString().getBytes();
           scan.addColumn(family, qualifier);
         } else {
@@ -96,7 +101,7 @@ public class HBaseRecordReader implements RecordReader {
       if (this.columns == null) {
         this.columns = Lists.newArrayList();
       }
-      rowKeySchemaPath = new SchemaPath(ROW_KEY);
+      rowKeySchemaPath = ROW_KEY_PATH;
       this.columns.add(rowKeySchemaPath);
     }
 
@@ -127,7 +132,8 @@ public class HBaseRecordReader implements RecordReader {
           MaterializedField field = MaterializedField.create(column, Types.optional(TypeProtos.MinorType.VARBINARY));
           NullableVarBinaryVector v = new NullableVarBinaryVector(field, context.getAllocator());
           output.addField(v);
-          vvMap.put(new FamilyQualifierWrapper(column.getPath().toString()), v);
+          String fullyQualified = column.getRootSegment().getPath() + "." + column.getRootSegment().getChild().getNameSegment().getPath();
+          vvMap.put(new FamilyQualifierWrapper(fullyQualified), v);
         }
       } catch (SchemaChangeException e) {
         throw new ExecutionSetupException(e);
@@ -206,7 +212,7 @@ public class HBaseRecordReader implements RecordReader {
   }
 
   private NullableVarBinaryVector addNewVector(String column) {
-    MaterializedField field = MaterializedField.create(new SchemaPath(column, ExpressionPosition.UNKNOWN), Types.optional(TypeProtos.MinorType.VARBINARY));
+    MaterializedField field = MaterializedField.create(SchemaPath.getCompoundPath(column.split("\\.")), Types.optional(TypeProtos.MinorType.VARBINARY));
     NullableVarBinaryVector v = new NullableVarBinaryVector(field, context.getAllocator());
     VectorAllocator.getAllocator(v, 100).alloc(TARGET_RECORD_COUNT);
     vvMap.put(new FamilyQualifierWrapper(column), v);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
index 1376210..991685c 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseSchemaFactory.java
@@ -26,6 +26,7 @@ import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.planner.logical.DynamicDrillTable;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
 import org.apache.hadoop.hbase.HTableDescriptor;
@@ -38,18 +39,17 @@ public class HBaseSchemaFactory implements SchemaFactory {
 
   final String schemaName;
   final HBaseStoragePlugin plugin;
-  
+
   public HBaseSchemaFactory(HBaseStoragePlugin plugin, String name) throws IOException {
     this.plugin = plugin;
     this.schemaName = name;
   }
 
   @Override
-  public Schema add(SchemaPlus parent) {
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
     HBaseSchema schema = new HBaseSchema(schemaName);
     SchemaPlus hPlus = parent.add(schemaName, schema);
     schema.setHolder(hPlus);
-    return schema;
   }
 
   class HBaseSchema extends AbstractSchema {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
index 1fab650..a82c6c3 100644
--- a/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
+++ b/contrib/storage-hbase/src/main/java/org/apache/drill/exec/store/hbase/HBaseStoragePlugin.java
@@ -19,11 +19,10 @@ package org.apache.drill.exec.store.hbase;
 
 import java.io.IOException;
 
-import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
 import org.apache.drill.common.JSONOptions;
-import org.apache.drill.common.logical.StoragePluginConfig;
+import org.apache.drill.exec.rpc.user.DrillUser;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.AbstractStoragePlugin;
 
@@ -63,8 +62,8 @@ public class HBaseStoragePlugin extends AbstractStoragePlugin {
   }
 
   @Override
-  public Schema createAndAddSchema(SchemaPlus parent) {
-    return schemaFactory.add(parent);
+  public void registerSchemas(DrillUser user, SchemaPlus parent) {
+    schemaFactory.registerSchemas(user, parent);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/distribution/src/resources/drill-env.sh
----------------------------------------------------------------------
diff --git a/distribution/src/resources/drill-env.sh b/distribution/src/resources/drill-env.sh
index 02a7022..d9fdb6f 100644
--- a/distribution/src/resources/drill-env.sh
+++ b/distribution/src/resources/drill-env.sh
@@ -13,4 +13,6 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 DRILL_MAX_DIRECT_MEMORY="8G"
-export DRILL_JAVA_OPTS="-XX:MaxDirectMemorySize=$DRILL_MAX_DIRECT_MEMORY"
+DRILL_INIT_HEAP="1g"
+DRILL_MAX_HEAP="4g"
+export DRILL_JAVA_OPTS="-Xms$DRILL_INIT_HEAP -Xmx$DRILL_MAX_HEAP -XX:MaxDirectMemorySize=$DRILL_MAX_DIRECT_MEMORY"

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
index 1297cb3..73f2e29 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DrillClient.java
@@ -59,7 +59,7 @@ import com.google.common.util.concurrent.SettableFuture;
  */
 public class DrillClient implements Closeable, ConnectionThrottle{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillClient.class);
-  
+
   DrillConfig config;
   private UserClient client;
   private volatile ClusterCoordinator clusterCoordinator;
@@ -67,7 +67,8 @@ public class DrillClient implements Closeable, ConnectionThrottle{
   private final TopLevelAllocator allocator = new TopLevelAllocator(Long.MAX_VALUE);
   private int reconnectTimes;
   private int reconnectDelay;
-  
+  private final boolean ownsZkConnection;
+
   public DrillClient() {
     this(DrillConfig.create());
   }
@@ -79,25 +80,26 @@ public class DrillClient implements Closeable, ConnectionThrottle{
   public DrillClient(DrillConfig config) {
     this(config, null);
   }
-  
+
   public DrillClient(DrillConfig config, ClusterCoordinator coordinator){
+    this.ownsZkConnection = coordinator == null;
     this.config = config;
     this.clusterCoordinator = coordinator;
     this.reconnectTimes = config.getInt(ExecConstants.BIT_RETRY_TIMES);
     this.reconnectDelay = config.getInt(ExecConstants.BIT_RETRY_DELAY);
   }
-  
+
   public DrillConfig getConfig(){
     return config;
   }
-  
+
   @Override
   public void setAutoRead(boolean enableAutoRead) {
     client.setAutoRead(enableAutoRead);
   }
 
-  
-  
+
+
   /**
    * Connects the client to a Drillbit server
    *
@@ -106,11 +108,11 @@ public class DrillClient implements Closeable, ConnectionThrottle{
   public void connect() throws RpcException {
     connect((String) null);
   }
-    
+
   public synchronized void connect(String connect) throws RpcException {
     if (connected) return;
 
-    if (clusterCoordinator == null) {
+    if (ownsZkConnection) {
       try {
         this.clusterCoordinator = new ZKClusterCoordinator(this.config, connect);
         this.clusterCoordinator.start(10000);
@@ -171,6 +173,16 @@ public class DrillClient implements Closeable, ConnectionThrottle{
    */
   public void close(){
     this.client.close();
+    if(ownsZkConnection){
+      try {
+        this.clusterCoordinator.close();
+      } catch (IOException e) {
+        logger.warn("Error while closing Cluster Coordinator.", e);
+      }
+    }
+
+    // TODO: fix tests that fail when this is called.
+    //allocator.close();
     connected = false;
   }
 
@@ -187,12 +199,12 @@ public class DrillClient implements Closeable, ConnectionThrottle{
     client.submitQuery(listener,query);
     return listener.getResults();
   }
-  
+
   public void cancelQuery(QueryId id){
     client.send(RpcType.CANCEL_QUERY, id, Ack.class);
   }
-  
-  
+
+
   /**
    * Submits a Logical plan for direct execution (bypasses parsing)
    *
@@ -203,7 +215,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
   public void runQuery(QueryType type, String plan, UserResultsListener resultsListener){
     client.submitQuery(resultsListener, newBuilder().setResultsMode(STREAM_FULL).setType(type).setPlan(plan).build());
   }
-  
+
   private class ListHoldingResultsListener implements UserResultsListener {
     private Vector<QueryResultBatch> results = new Vector<QueryResultBatch>();
     private SettableFuture<List<QueryResultBatch>> future = SettableFuture.create();
@@ -245,7 +257,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
         future.set(results);
       }
     }
-  
+
     public List<QueryResultBatch> getResults() throws RpcException{
       try{
         return future.get();
@@ -258,7 +270,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
     public void queryIdArrived(QueryId queryId) {
     }
   }
-  
+
   private class FutureHandler extends AbstractCheckedFuture<Void, RpcException> implements RpcConnectionHandler<ServerConnection>, DrillRpcFuture<Void>{
 
     protected FutureHandler() {
@@ -278,7 +290,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
     private SettableFuture<Void> getInner(){
       return (SettableFuture<Void>) delegate();
     }
-    
+
     @Override
     protected RpcException mapException(Exception e) {
       return RpcException.mapException(e);
@@ -288,7 +300,7 @@ public class DrillClient implements Closeable, ConnectionThrottle{
     public ByteBuf getBuffer() {
       return null;
     }
-    
+
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
index cfffed9..c69a43e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/DumpCat.java
@@ -68,7 +68,7 @@ public class DumpCat {
       System.out.println(String.format("Trace file %s not created", o.location));
       System.exit(-1);
     }
- 
+
     FileInputStream input = new FileInputStream(file.getAbsoluteFile());
 
     if (o.batch < 0) {
@@ -142,11 +142,11 @@ public class DumpCat {
       String avgRecSizeStr = null;
       if (this.rows>0)
         avgRecSizeStr = String.format("Average Record Size : %d ", this.dataSize/this.rows);
-      else 
+      else
         avgRecSizeStr = "Average Record Size : 0";
-      
+
       return String.format("Records : %d / %d \n", this.selectedRows, this.rows) +
-             avgRecSizeStr + 
+             avgRecSizeStr +
              String.format("\n Total Data Size : %d", this.dataSize);
     }
   }
@@ -188,7 +188,7 @@ public class DumpCat {
 
        prevSchema = vectorContainer.getSchema();
        batchNum ++;
- 
+
        vectorContainer.zeroVectors();
     }
 
@@ -253,7 +253,7 @@ public class DumpCat {
       for (VectorWrapper w : vectorContainer) {
         MaterializedField field = w.getValueVector().getField();
         System.out.println (String.format("name : %s, minor_type : %s, data_mode : %s",
-                                          field.getName(),
+                                          field.toExpr(),
                                           field.getType().getMinorType().toString(),
                                           field.isNullable() ? "nullable":"non-nullable"
                           ));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
new file mode 100644
index 0000000..f161782
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/PrintingResultsListener.java
@@ -0,0 +1,100 @@
+/**
+ * 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.client;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import org.apache.drill.exec.client.QuerySubmitter.Format;
+import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
+import org.apache.drill.exec.memory.TopLevelAllocator;
+import org.apache.drill.exec.proto.UserBitShared.QueryId;
+import org.apache.drill.exec.record.RecordBatchLoader;
+import org.apache.drill.exec.rpc.RpcException;
+import org.apache.drill.exec.rpc.user.ConnectionThrottle;
+import org.apache.drill.exec.rpc.user.QueryResultBatch;
+import org.apache.drill.exec.rpc.user.UserResultsListener;
+import org.apache.drill.exec.util.VectorUtil;
+
+public class PrintingResultsListener implements UserResultsListener {
+  AtomicInteger count = new AtomicInteger();
+  private CountDownLatch latch = new CountDownLatch(1);
+  RecordBatchLoader loader;
+  Format format;
+  int    columnWidth;
+  BufferAllocator allocator = new TopLevelAllocator();
+  volatile Exception exception;
+
+  public PrintingResultsListener(Format format, int columnWidth) {
+    loader = new RecordBatchLoader(allocator);
+    this.format = format;
+    this.columnWidth = columnWidth;
+  }
+
+  @Override
+  public void submissionFailed(RpcException ex) {
+    exception = ex;
+    latch.countDown();
+  }
+
+  @Override
+  public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
+    int rows = result.getHeader().getRowCount();
+    if (result.getData() != null) {
+      count.addAndGet(rows);
+      try {
+        loader.load(result.getHeader().getDef(), result.getData());
+      } catch (SchemaChangeException e) {
+        submissionFailed(new RpcException(e));
+      }
+
+      switch(format) {
+        case TABLE:
+          VectorUtil.showVectorAccessibleContent(loader, columnWidth);
+          break;
+        case TSV:
+          VectorUtil.showVectorAccessibleContent(loader, "\t");
+          break;
+        case CSV:
+          VectorUtil.showVectorAccessibleContent(loader, ",");
+          break;
+      }
+
+    }
+
+    boolean isLastChunk = result.getHeader().getIsLastChunk();
+    result.release();
+
+    if (isLastChunk) {
+      allocator.close();
+      latch.countDown();
+    }
+
+  }
+
+  public int await() throws Exception {
+    latch.await();
+    if(exception != null) throw exception;
+    return count.get();
+  }
+
+  @Override
+  public void queryIdArrived(QueryId queryId) {
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
index 4551e53..0234b7e 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/client/QuerySubmitter.java
@@ -20,24 +20,11 @@ package org.apache.drill.exec.client;
 import java.nio.ByteBuffer;
 import java.nio.file.Files;
 import java.nio.file.Paths;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
 
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.coord.ZKClusterCoordinator;
-import org.apache.drill.exec.exception.SchemaChangeException;
-import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.proto.UserProtos;
-import org.apache.drill.exec.record.RecordBatchLoader;
-import org.apache.drill.exec.rpc.RpcException;
-import org.apache.drill.exec.rpc.user.ConnectionThrottle;
-import org.apache.drill.exec.rpc.user.QueryResultBatch;
-import org.apache.drill.exec.rpc.user.UserResultsListener;
-import org.apache.drill.exec.server.BootStrapContext;
 import org.apache.drill.exec.server.Drillbit;
 import org.apache.drill.exec.server.RemoteServiceSet;
 import org.apache.drill.exec.util.VectorUtil;
@@ -46,6 +33,7 @@ import com.beust.jcommander.JCommander;
 import com.beust.jcommander.Parameter;
 import com.beust.jcommander.ParameterException;
 import com.google.common.base.Charsets;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Stopwatch;
 
 public class QuerySubmitter {
@@ -115,10 +103,13 @@ public class QuerySubmitter {
     Preconditions.checkArgument(!(planLocation == null && queryString == null), "Must provide either query file or query string");
     Preconditions.checkArgument(!(planLocation != null && queryString != null), "Must provide either query file or query string, not both");
 
+    RemoteServiceSet serviceSet = null;
+    Drillbit[] drillbits = null;
+
     try{
       if (local) {
-        RemoteServiceSet serviceSet = RemoteServiceSet.getLocalServiceSet();
-        Drillbit[] drillbits = new Drillbit[bits];
+        serviceSet = RemoteServiceSet.getLocalServiceSet();
+        drillbits = new Drillbit[bits];
         for (int i = 0; i < bits; i++) {
           drillbits[i] = new Drillbit(config, serviceSet);
           drillbits[i].run();
@@ -130,126 +121,81 @@ public class QuerySubmitter {
         client = new DrillClient(config, clusterCoordinator);
       }
       client.connect();
-      QueryResultsListener listener;
+
       String plan;
       if (queryString == null) {
         plan = Charsets.UTF_8.decode(ByteBuffer.wrap(Files.readAllBytes(Paths.get(planLocation)))).toString();
       } else {
         plan = queryString;
       }
-      String[] queries;
-      UserProtos.QueryType queryType;
-      type = type.toLowerCase();
-      switch(type) {
-        case "sql":
-          queryType = UserProtos.QueryType.SQL;
-          queries = plan.trim().split(";");
-          break;
-        case "logical":
-          queryType = UserProtos.QueryType.LOGICAL;
-          queries = new String[]{ plan };
-          break;
-        case "physical":
-          queryType = UserProtos.QueryType.PHYSICAL;
-          queries = new String[]{ plan };
-          break;
-        default:
-          System.out.println("Invalid query type: " + type);
-          return -1;
-      }
-      Format outputFormat;
-      format = format.toLowerCase();
-      switch(format) {
-        case "csv":
-          outputFormat = Format.CSV;
-          break;
-        case "tsv":
-          outputFormat = Format.TSV;
-          break;
-        case "table":
-          outputFormat = Format.TABLE;
-          break;
-        default:
-          System.out.println("Invalid format type: " + format);
-          return -1;
-      }
-      Stopwatch watch = new Stopwatch();
-      for (String query : queries) {
-        listener = new QueryResultsListener(outputFormat, width);
-        watch.start();
-        client.runQuery(queryType, query, listener);
-        int rows = listener.await();
-        System.out.println(String.format("%d record%s selected (%f seconds)", rows, rows > 1 ? "s" : "", (float) watch.elapsed(TimeUnit.MILLISECONDS) / (float) 1000));
-        if (query != queries[queries.length - 1]) {
-          System.out.println();
-        }
-        watch.stop();
-        watch.reset();
-      }
-      return 0;
+      return submitQuery(client, plan, type, format, width);
+
     }finally{
       if(client != null) client.close();
+      if(local){
+        for(Drillbit b : drillbits){
+          b.close();
+        }
+        serviceSet.close();
+      }
     }
   }
 
-  private class QueryResultsListener implements UserResultsListener {
-    AtomicInteger count = new AtomicInteger();
-    private CountDownLatch latch = new CountDownLatch(1);
-    RecordBatchLoader loader = new RecordBatchLoader(new BootStrapContext(DrillConfig.create()).getAllocator());
-    Format format;
-    int    columnWidth;
-    volatile Exception exception;
-
-    public QueryResultsListener(Format format, int columnWidth) {
-      this.format = format;
-      this.columnWidth = columnWidth;
+  public int submitQuery(DrillClient client, String plan, String type, String format, int width) throws Exception {
+
+    PrintingResultsListener listener;
+
+    String[] queries;
+    UserProtos.QueryType queryType;
+    type = type.toLowerCase();
+    switch(type) {
+      case "sql":
+        queryType = UserProtos.QueryType.SQL;
+        queries = plan.trim().split(";");
+        break;
+      case "logical":
+        queryType = UserProtos.QueryType.LOGICAL;
+        queries = new String[]{ plan };
+        break;
+      case "physical":
+        queryType = UserProtos.QueryType.PHYSICAL;
+        queries = new String[]{ plan };
+        break;
+      default:
+        System.out.println("Invalid query type: " + type);
+        return -1;
     }
 
-    @Override
-    public void submissionFailed(RpcException ex) {
-      exception = ex;
-      latch.countDown();
+    Format outputFormat;
+    format = format.toLowerCase();
+    switch(format) {
+      case "csv":
+        outputFormat = Format.CSV;
+        break;
+      case "tsv":
+        outputFormat = Format.TSV;
+        break;
+      case "table":
+        outputFormat = Format.TABLE;
+        break;
+      default:
+        System.out.println("Invalid format type: " + format);
+        return -1;
     }
-
-    @Override
-    public void resultArrived(QueryResultBatch result, ConnectionThrottle throttle) {
-      int rows = result.getHeader().getRowCount();
-      if (result.getData() != null) {
-        count.addAndGet(rows);
-        try {
-          loader.load(result.getHeader().getDef(), result.getData());
-        } catch (SchemaChangeException e) {
-          submissionFailed(new RpcException(e));
-        }
-
-        switch(format) {
-          case TABLE:
-            VectorUtil.showVectorAccessibleContent(loader, columnWidth);
-            break;
-          case TSV:
-            VectorUtil.showVectorAccessibleContent(loader, "\t");
-            break;
-          case CSV:
-            VectorUtil.showVectorAccessibleContent(loader, ",");
-            break;
-        }
-        
+    Stopwatch watch = new Stopwatch();
+    for (String query : queries) {
+      listener = new PrintingResultsListener(outputFormat, width);
+      watch.start();
+      client.runQuery(queryType, query, listener);
+      int rows = listener.await();
+      System.out.println(String.format("%d record%s selected (%f seconds)", rows, rows > 1 ? "s" : "", (float) watch.elapsed(TimeUnit.MILLISECONDS) / (float) 1000));
+      if (query != queries[queries.length - 1]) {
+        System.out.println();
       }
-      
-      if (result.getHeader().getIsLastChunk()) {
-        latch.countDown();
-      }
-      result.release();
-    }
-
-    public int await() throws Exception {
-      latch.await();
-      if(exception != null) throw exception;
-      return count.get();
+      watch.stop();
+      watch.reset();
     }
+    return 0;
 
-    @Override
-    public void queryIdArrived(QueryId queryId) {
-    }
   }
 }


[45/51] [abbrv] Move to Optiq 0.6 Also includes: -improve exception catching -move schema path parsing to Antlr -close zookeeper connection on if client created -enhance BaseTestQuery and have other query tests utilize it -Various test fixes for better m

Posted by ja...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
index 080679b..30cde91 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/compile/MergeAdapter.java
@@ -50,14 +50,14 @@ import com.google.common.io.Files;
  * methods and fields of the class to merge to the class that is being visited.
  */
 class MergeAdapter extends ClassVisitor {
-  
+
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(MergeAdapter.class);
-  
+
   private ClassNode classToMerge;
   private ClassSet set;
-  
+
   private Set<String> mergingNames = Sets.newHashSet();
-  
+
   private MergeAdapter(ClassSet set, ClassVisitor cv, ClassNode cn) {
     super(Opcodes.ASM4, cv);
     this.classToMerge = cn;
@@ -88,16 +88,16 @@ class MergeAdapter extends ClassVisitor {
     System.out.println("Annotation");
     return super.visitAnnotation(desc, visible);
   }
-  
+
   // visit the class
   public void visit(int version, int access, String name, String signature, String superName, String[] interfaces) {
     // use the access and names of the impl class.
     if(name.contains("$")){
       super.visit(version, access, name, signature, superName, interfaces);
     }else{
-      super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL, name, signature, superName, interfaces);  
+      super.visit(version, access ^ Modifier.ABSTRACT | Modifier.FINAL, name, signature, superName, interfaces);
     }
-    
+
 //    this.cname = name;
   }
 
@@ -107,8 +107,8 @@ class MergeAdapter extends ClassVisitor {
 
     // skip all abstract methods as they should have implementations.
     if ((access & Modifier.ABSTRACT) != 0 || mergingNames.contains(arg1)) {
-      
-      logger.debug("Skipping copy of '{}()' since it is abstract or listed elsewhere.", arg1);
+
+//      logger.debug("Skipping copy of '{}()' since it is abstract or listed elsewhere.", arg1);
       return null;
     }
     if(arg3 != null){
@@ -156,8 +156,8 @@ class MergeAdapter extends ClassVisitor {
   public FieldVisitor visitField(int access, String name, String desc, String signature, Object value) {
     return super.visitField(access, name, desc, signature, value);
   }
-  
-  
+
+
   public static class MergedClassResult{
     public byte[] bytes;
     public Collection<String> innerClasses;
@@ -166,32 +166,32 @@ class MergeAdapter extends ClassVisitor {
       this.bytes = bytes;
       this.innerClasses = innerClasses;
     }
-    
-    
+
+
   }
-  
+
   public static MergedClassResult getMergedClass(ClassSet set, byte[] precompiledClass, ClassNode generatedClass) throws IOException{
 
     // Setup adapters for merging, remapping class names and class writing. This is done in reverse order of how they
     // will be evaluated.
-    
+
     ClassWriter cw = new ClassWriter(ClassWriter.COMPUTE_FRAMES);
     RemapClasses re = new RemapClasses(set);
     ClassVisitor remappingAdapter = new RemappingClassAdapter(cw, re);
     ClassVisitor visitor = remappingAdapter;
     if(generatedClass != null){
-      visitor = new MergeAdapter(set, remappingAdapter, generatedClass);  
+      visitor = new MergeAdapter(set, remappingAdapter, generatedClass);
     }
     ClassReader tReader = new ClassReader(precompiledClass);
     tReader.accept(visitor, ClassReader.EXPAND_FRAMES);
     byte[] outputClass = cw.toByteArray();
-    
+
     // enable when you want all the generated merged class files to also be written to disk.
     //Files.write(outputClass, new File(String.format("/tmp/drill-generated-classes/%s-output.class", set.generated.dot)));
 
     return new MergedClassResult(outputClass, re.getInnerClasses());
   }
-  
+
 
   static class RemapClasses extends Remapper {
 
@@ -208,15 +208,15 @@ class MergeAdapter extends ClassVisitor {
 
     @Override
     public String map(String typeName) {
-      
+
       // remap the names of all classes that start with the old class name.
       if (typeName.startsWith(top.precompiled.slash)) {
-        
+
         // write down all the sub classes.
         if (typeName.startsWith(current.precompiled.slash + "$")){
           innerClasses.add(typeName);
         }
-          
+
         return typeName.replace(top.precompiled.slash, top.generated.slash);
       }
       return typeName;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java
index 4221664..2728759 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/Accountor.java
@@ -39,7 +39,7 @@ public class Accountor {
   private final long total;
   private ConcurrentMap<ByteBuf, DebugStackTrace> buffers = Maps.newConcurrentMap();
   private final FragmentHandle handle;
-  
+
   public Accountor(FragmentHandle handle, Accountor parent, long max, long preAllocated) {
     // TODO: fix preallocation stuff
     AtomicRemainder parentRemainder = parent != null ? parent.remainder : null;
@@ -106,7 +106,7 @@ public class Accountor {
       }
       sb.append(".\n");
 
-      
+
       Multimap<DebugStackTrace, DebugStackTrace> multi = LinkedListMultimap.create();
       for (DebugStackTrace t : buffers.values()) {
         multi.put(t, t);
@@ -114,7 +114,7 @@ public class Accountor {
 
       for (DebugStackTrace entry : multi.keySet()) {
         Collection<DebugStackTrace> allocs = multi.get(entry);
-        
+
         sb.append("\n\n\tTotal ");
         sb.append(allocs.size());
         sb.append(" allocation(s) of byte size(s): ");
@@ -127,20 +127,21 @@ public class Accountor {
           }
           sb.append(", ");
         }
-        
+
         sb.append("at stack location:\n");
         entry.addToString(sb);
       }
-      
+
       throw new IllegalStateException(sb.toString());
-      
+
     }
 
-    
+    remainder.close();
+
   }
 
   private class DebugStackTrace {
-    
+
     private StackTraceElement[] elements;
     private long size;
     private String desc;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
index 927e5a2..95e57d2 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/memory/AtomicRemainder.java
@@ -20,8 +20,8 @@ package org.apache.drill.exec.memory;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
- * 
- * 
+ *
+ *
  * TODO: Fix this so that preallocation can never be released back to general pool until allocator is closed.
  */
 public class AtomicRemainder {
@@ -32,6 +32,7 @@ public class AtomicRemainder {
   private final AtomicLong unaccountable;
   private final long max;
   private final long pre;
+  private boolean closed = false;
 
   public AtomicRemainder(AtomicRemainder parent, long max, long pre) {
     this.parent = parent;
@@ -52,7 +53,7 @@ public class AtomicRemainder {
   /**
    * Automatically allocate memory. This is used when an actual allocation happened to be larger than requested. This
    * memory has already been used up so it must be accurately accounted for in future allocations.
-   * 
+   *
    * @param size
    */
   public void forceGet(long size) {
@@ -101,7 +102,7 @@ public class AtomicRemainder {
 
   /**
    * Return the memory accounting to the allocation pool. Make sure to first maintain hold of the preallocated memory.
-   * 
+   *
    * @param size
    */
   public void returnAllocation(long size) {
@@ -115,4 +116,10 @@ public class AtomicRemainder {
     }
   }
 
+  public void close(){
+    if(!closed){
+      closed = true;
+//      if(parent != null) parent.returnAllocation(pre);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
index e8ab192..ddc5025 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/FragmentContext.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 
 import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.tools.Frameworks;
 
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.compile.ClassTransformer;
@@ -96,9 +97,11 @@ public class FragmentContext implements Closeable {
   public DrillbitContext getDrillbitContext() {
     return context;
   }
-  
+
   public SchemaPlus getRootSchema(){
-    return context.getStorage().getSchemaFactory().getOrphanedRootSchema();
+    SchemaPlus root = Frameworks.createRootSchema();
+    context.getStorage().getSchemaFactory().registerSchemas(null, root);
+    return root;
   }
 
   /**
@@ -116,7 +119,7 @@ public class FragmentContext implements Closeable {
   public long getQueryStartTime() {
       return this.queryStartTime;
   }
-  
+
   /**
    * The FragmentHandle for this Fragment
    * @return FragmentHandle
@@ -136,10 +139,10 @@ public class FragmentContext implements Closeable {
   public <T> T getImplementationClass(ClassGenerator<T> cg) throws ClassTransformationException, IOException {
     return getImplementationClass(cg.getCodeGenerator());
   }
-  
+
   public <T> T getImplementationClass(CodeGenerator<T> cg) throws ClassTransformationException, IOException {
     long t1 = System.nanoTime();
-    
+
     T t = transformer.getImplementationClass(this.loader, cg.getDefinition(), cg.generate(),
         cg.getMaterializedClassName());
     logger.debug("Compile time: {} millis.", (System.nanoTime() - t1) / 1000 / 1000);
@@ -177,7 +180,7 @@ public class FragmentContext implements Closeable {
   public void addDaemonThread(Thread thread) {
     daemonThreads.add(thread);
     thread.start();
-    
+
   }
 
   public IncomingBuffers getBuffers() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.java
new file mode 100644
index 0000000..7e6ae8e
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/Multitimer.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.ops;
+
+import org.slf4j.Logger;
+
+public class Multitimer<T extends Enum<T>> {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(Multitimer.class);
+
+  private final long start;
+  private final long[] times;
+  private final Class<T> clazz;
+
+  public Multitimer(Class<T> clazz){
+    this.times = new long[clazz.getEnumConstants().length];
+    this.start = System.nanoTime();
+    this.clazz = clazz;
+  }
+
+  public void mark(T timer){
+    times[timer.ordinal()] = System.nanoTime();
+  }
+
+  public void log(Logger logger){
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
index 1564b09..17d47aa 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QueryContext.java
@@ -19,6 +19,9 @@ package org.apache.drill.exec.ops;
 
 import java.util.Collection;
 
+import net.hydromatic.optiq.SchemaPlus;
+import net.hydromatic.optiq.tools.Frameworks;
+
 import org.apache.drill.common.config.DrillConfig;
 import org.apache.drill.exec.cache.DistributedCache;
 import org.apache.drill.exec.expr.fn.FunctionImplementationRegistry;
@@ -27,28 +30,48 @@ import org.apache.drill.exec.proto.CoordinationProtos.DrillbitEndpoint;
 import org.apache.drill.exec.proto.UserBitShared.QueryId;
 import org.apache.drill.exec.rpc.control.WorkEventBus;
 import org.apache.drill.exec.rpc.data.DataConnectionCreator;
+import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.server.DrillbitContext;
 import org.apache.drill.exec.store.StoragePluginRegistry;
-import org.apache.drill.exec.store.StoragePluginRegistry.DrillSchemaFactory;
 
 public class QueryContext {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(QueryContext.class);
-  
-  private QueryId queryId;
-  private DrillbitContext drillbitContext;
-  private WorkEventBus workBus;
-  
-  public QueryContext(QueryId queryId, DrillbitContext drllbitContext) {
+
+  private final QueryId queryId;
+  private final DrillbitContext drillbitContext;
+  private final WorkEventBus workBus;
+  private UserSession session;
+  public final Multitimer<QuerySetup> timer;
+
+  public QueryContext(UserSession session, QueryId queryId, DrillbitContext drllbitContext) {
     super();
     this.queryId = queryId;
     this.drillbitContext = drllbitContext;
     this.workBus = drllbitContext.getWorkBus();
+    this.session = session;
+    this.timer = new Multitimer<>(QuerySetup.class);
+  }
+
+  public UserSession getSession(){
+    return session;
   }
-  
+
+  public SchemaPlus getNewDefaultSchema(){
+    SchemaPlus rootSchema = Frameworks.createRootSchema();
+    drillbitContext.getSchemaFactory().registerSchemas(session.getUser(), rootSchema);
+    SchemaPlus defaultSchema = session.getDefaultSchema(rootSchema);
+    if(defaultSchema == null){
+      return rootSchema;
+    }else{
+      return defaultSchema;
+    }
+  }
+
+
   public DrillbitEndpoint getCurrentEndpoint(){
     return drillbitContext.getEndpoint();
   }
-  
+
   public QueryId getQueryId() {
     return queryId;
   }
@@ -56,36 +79,32 @@ public class QueryContext {
   public StoragePluginRegistry getStorage(){
     return drillbitContext.getStorage();
   }
-  
-  
+
+
   public DistributedCache getCache(){
     return drillbitContext.getCache();
   }
-  
+
   public Collection<DrillbitEndpoint> getActiveEndpoints(){
     return drillbitContext.getBits();
   }
-  
+
   public PhysicalPlanReader getPlanReader(){
     return drillbitContext.getPlanReader();
   }
-  
+
   public DataConnectionCreator getDataConnectionsPool(){
     return drillbitContext.getDataConnectionsPool();
   }
-  
+
   public DrillConfig getConfig(){
     return drillbitContext.getConfig();
   }
-  
+
   public WorkEventBus getWorkBus(){
     return workBus;
   }
 
-  public DrillSchemaFactory getFactory(){
-    return drillbitContext.getSchemaFactory();
-  }
-  
   public FunctionImplementationRegistry getFunctionRegistry(){
     return drillbitContext.getFunctionImplementationRegistry();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
new file mode 100644
index 0000000..ef73867
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/QuerySetup.java
@@ -0,0 +1,26 @@
+/**
+ * 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.ops;
+
+public enum QuerySetup {
+  START_SQL_PARSING,
+  START_SQL_VALIDATION,
+  START_SQL_TO_REL,
+  START_OPTIQ_REL_TO_DRILL_LOGICAL,
+  START_DRILL_LOGICAL_TO_PHYSICAL;
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
index 6c111bb..38d56ea 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/OutputMutator.java
@@ -18,11 +18,15 @@
 package org.apache.drill.exec.physical.impl;
 
 import org.apache.drill.exec.exception.SchemaChangeException;
+import org.apache.drill.exec.memory.BufferAllocator;
 import org.apache.drill.exec.record.MaterializedField;
 import org.apache.drill.exec.vector.ValueVector;
 
 public interface OutputMutator {
   public void removeField(MaterializedField field) throws SchemaChangeException;
+  public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException ;
+
+  @Deprecated
   public void addField(ValueVector vector) throws SchemaChangeException ;
   public void removeAllFields();
   public void setNewSchema() throws SchemaChangeException;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
index 24ea9c4..ace2677 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/ScanBatch.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import org.apache.drill.common.exceptions.ExecutionSetupException;
 import org.apache.drill.common.expression.SchemaPath;
 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.record.BatchSchema;
 import org.apache.drill.exec.record.BatchSchema.SelectionVectorMode;
@@ -168,6 +169,15 @@ public class ScanBatch implements RecordBatch {
       ScanBatch.this.schemaChanged = true;
     }
 
+    @SuppressWarnings("unchecked")
+    @Override
+    public <T extends ValueVector> T addField(MaterializedField field, Class<T> clazz) throws SchemaChangeException {
+      ValueVector v = TypeHelper.getNewVector(field, context.getAllocator());
+      if(!clazz.isAssignableFrom(v.getClass())) throw new SchemaChangeException(String.format("The class that was provided %s does not correspond to the expected vector type of %s.", clazz.getSimpleName(), v.getClass().getSimpleName()));
+      addField(v);
+      return (T) v;
+    }
+
   }
 
   @Override
@@ -179,7 +189,7 @@ public class ScanBatch implements RecordBatch {
   public WritableBatch getWritableBatch() {
     return WritableBatch.get(this);
   }
-  
+
   public void cleanup(){
     container.clear();
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
index 569bfc7..d2d8d30 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/TopN/PriorityQueueTemplate.java
@@ -33,7 +33,7 @@ import java.util.concurrent.TimeUnit;
 
 public abstract class PriorityQueueTemplate implements PriorityQueue {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(PriorityQueueTemplate.class);
-  
+
   private SelectionVector4 heapSv4;//This holds the heap
   private SelectionVector4 finalSv4;//This is for final sorted output
   private ExpandableHyperContainer hyperBatch;
@@ -58,7 +58,7 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
     BatchSchema schema = container.getSchema();
     VectorContainer newContainer = new VectorContainer();
     for (MaterializedField field : schema) {
-      int id = container.getValueVectorId(new SchemaPath(field.getName(), ExpressionPosition.UNKNOWN)).getFieldId();
+      int id = container.getValueVectorId(field.getAsSchemaPath()).getFieldId();
       newContainer.add(container.getValueAccessorById(id, field.getValueClass()).getValueVectors());
     }
     newContainer.buildSchema(BatchSchema.SelectionVectorMode.FOUR_BYTE);
@@ -72,7 +72,7 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
     }
     v4.clear();
   }
-  
+
   @Override
   public void add(FragmentContext context, RecordBatchData batch) throws SchemaChangeException{
     Stopwatch watch = new Stopwatch();
@@ -184,13 +184,13 @@ public abstract class PriorityQueueTemplate implements PriorityQueue {
     siftDown();
     return value;
   }
-  
+
   public void swap(int sv0, int sv1) {
     int tmp = heapSv4.get(sv0);
     heapSv4.set(sv0, heapSv4.get(sv1));
     heapSv4.set(sv1, tmp);
   }
-  
+
   public int compare(int leftIndex, int rightIndex) {
     int sv1 = heapSv4.get(leftIndex);
     int sv2 = heapSv4.get(rightIndex);

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
index 5a7a6fa..775766d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/common/HashTableTemplate.java
@@ -44,7 +44,7 @@ import org.apache.drill.exec.vector.BigIntVector;
 import org.apache.drill.exec.expr.holders.BigIntHolder;
 import org.apache.drill.exec.vector.allocator.FixedVectorAllocator;
 
-public abstract class HashTableTemplate implements HashTable { 
+public abstract class HashTableTemplate implements HashTable {
 
   private static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(HashTable.class);
   private static final boolean EXTRA_DEBUG = false;
@@ -72,7 +72,7 @@ public abstract class HashTableTemplate implements HashTable {
   private int freeIndex = 0;
 
   // Placeholder for the current index while probing the hash table
-  private IntHolder currentIdxHolder; 
+  private IntHolder currentIdxHolder;
 
   private FragmentContext context;
 
@@ -86,7 +86,7 @@ public abstract class HashTableTemplate implements HashTable {
   private RecordBatch outgoing;
 
   // Hash table configuration parameters
-  private HashTableConfig htConfig; 
+  private HashTableConfig htConfig;
 
   // The original container from which others may be cloned
   private VectorContainer htContainerOrig;
@@ -96,15 +96,15 @@ public abstract class HashTableTemplate implements HashTable {
   private int outputCount = 0;
 
   // This class encapsulates the links, keys and values for up to BATCH_SIZE
-  // *unique* records. Thus, suppose there are N incoming record batches, each 
-  // of size BATCH_SIZE..but they have M unique keys altogether, the number of 
+  // *unique* records. Thus, suppose there are N incoming record batches, each
+  // of size BATCH_SIZE..but they have M unique keys altogether, the number of
   // BatchHolders will be (M/BATCH_SIZE) + 1
   public class BatchHolder {
 
     // Container of vectors to hold type-specific keys
     private VectorContainer htContainer;
 
-    // Array of 'link' values 
+    // Array of 'link' values
     private IntVector links;
 
     // Array of hash values - this is useful when resizing the hash table
@@ -122,7 +122,7 @@ public abstract class HashTableTemplate implements HashTable {
           ValueVector vv = TypeHelper.getNewVector(w.getField(), context.getAllocator());
           VectorAllocator.getAllocator(vv, 50 /* avg width */).alloc(HashTable.BATCH_SIZE);
           htContainer.add(vv);
-        }      
+        }
       }
 
       links = allocMetadataVector(HashTable.BATCH_SIZE, EMPTY_SLOT);
@@ -145,10 +145,10 @@ public abstract class HashTableTemplate implements HashTable {
     }
 
     // Check if the key at the currentIdx position in hash table matches the key
-    // at the incomingRowIdx. if the key does not match, update the 
+    // at the incomingRowIdx. if the key does not match, update the
     // currentIdxHolder with the index of the next link.
-    private boolean isKeyMatch(int incomingRowIdx, 
-                               IntHolder currentIdxHolder, 
+    private boolean isKeyMatch(int incomingRowIdx,
+                               IntHolder currentIdxHolder,
                                boolean isProbe) {
 
       int currentIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
@@ -156,7 +156,7 @@ public abstract class HashTableTemplate implements HashTable {
 
       if (isProbe)
         match = isKeyMatchInternalProbe(incomingRowIdx, currentIdxWithinBatch);
-      else 
+      else
         match = isKeyMatchInternalBuild(incomingRowIdx, currentIdxWithinBatch);
 
       if (! match) {
@@ -165,9 +165,9 @@ public abstract class HashTableTemplate implements HashTable {
       return match;
     }
 
-    // Insert a new <key1, key2...keyN> entry coming from the incoming batch into the hash table 
-    // container at the specified index 
-    private boolean insertEntry(int incomingRowIdx, int currentIdx, int hashValue, BatchHolder lastEntryBatch, int lastEntryIdxWithinBatch) { 
+    // Insert a new <key1, key2...keyN> entry coming from the incoming batch into the hash table
+    // container at the specified index
+    private boolean insertEntry(int incomingRowIdx, int currentIdx, int hashValue, BatchHolder lastEntryBatch, int lastEntryIdxWithinBatch) {
       int currentIdxWithinBatch = currentIdx & BATCH_MASK;
 
       if (! setValue(incomingRowIdx, currentIdxWithinBatch)) {
@@ -179,7 +179,7 @@ public abstract class HashTableTemplate implements HashTable {
         lastEntryBatch.updateLinks(lastEntryIdxWithinBatch, currentIdx);
       }
 
-      // since this is the last entry in the hash chain, the links array at position currentIdx 
+      // since this is the last entry in the hash chain, the links array at position currentIdx
       // will point to a null (empty) slot
       links.getMutator().set(currentIdxWithinBatch, EMPTY_SLOT);
       hashValues.getMutator().set(currentIdxWithinBatch, hashValue);
@@ -204,7 +204,7 @@ public abstract class HashTableTemplate implements HashTable {
       IntVector newHashValues = allocMetadataVector(size, 0);
 
       for (int i = 0; i <= maxOccupiedIdx; i++) {
-        int entryIdxWithinBatch = i; 
+        int entryIdxWithinBatch = i;
         int entryIdx = entryIdxWithinBatch + batchStartIdx;
         int hash = hashValues.getAccessor().get(entryIdxWithinBatch); // get the already saved hash value
         int bucketIdx = getBucketIndex(hash, numbuckets);
@@ -237,35 +237,35 @@ public abstract class HashTableTemplate implements HashTable {
 
         }
 
-      }      
+      }
 
       links.clear();
       hashValues.clear();
-      
+
       links = newLinks;
       hashValues = newHashValues;
     }
-    
+
     private boolean outputKeys() {
 
-      /** for debugging 
+      /** for debugging
       Object tmp = (htContainer).getValueAccessorById(0, BigIntVector.class).getValueVector();
       BigIntVector vv0 = null;
       BigIntHolder holder = null;
 
-      if (tmp != null) { 
+      if (tmp != null) {
         vv0 = ((BigIntVector) tmp);
         holder = new BigIntHolder();
       }
       */
 
-      for (int i = 0; i <= maxOccupiedIdx; i++) { 
+      for (int i = 0; i <= maxOccupiedIdx; i++) {
         if (outputRecordKeys(i, outputCount) ) {
           if (EXTRA_DEBUG) logger.debug("Outputting keys to {}", outputCount) ;
 
-          // debugging 
+          // debugging
           // holder.value = vv0.getAccessor().get(i);
-          // if (holder.value == 100018 || holder.value == 100021) { 
+          // if (holder.value == 100018 || holder.value == 100021) {
           //  logger.debug("Outputting key = {} at index - {} to outgoing index = {}.", holder.value, i, outputCount);
           // }
 
@@ -288,41 +288,41 @@ public abstract class HashTableTemplate implements HashTable {
         }
       }
     }
-    
+
     private void clear() {
       htContainer.clear();;
       links.clear();
       hashValues.clear();
     }
 
-    // These methods will be code-generated 
+    // These methods will be code-generated
 
     @RuntimeOverridden
-    protected void setupInterior(@Named("incomingBuild") RecordBatch incomingBuild, 
+    protected void setupInterior(@Named("incomingBuild") RecordBatch incomingBuild,
                                  @Named("incomingProbe") RecordBatch incomingProbe,
                                  @Named("outgoing") RecordBatch outgoing,
                                  @Named("htContainer") VectorContainer htContainer) {}
 
     @RuntimeOverridden
-    protected boolean isKeyMatchInternalBuild(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;} 
+    protected boolean isKeyMatchInternalBuild(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;}
 
     @RuntimeOverridden
-    protected boolean isKeyMatchInternalProbe(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;} 
+    protected boolean isKeyMatchInternalProbe(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;}
 
     @RuntimeOverridden
-    protected boolean setValue(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;} 
+    protected boolean setValue(@Named("incomingRowIdx") int incomingRowIdx, @Named("htRowIdx") int htRowIdx) {return false;}
 
     @RuntimeOverridden
-    protected boolean outputRecordKeys(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {return false;} 
+    protected boolean outputRecordKeys(@Named("htRowIdx") int htRowIdx, @Named("outRowIdx") int outRowIdx) {return false;}
 
   } // class BatchHolder
 
 
   @Override
-  public void setup(HashTableConfig htConfig, FragmentContext context, 
+  public void setup(HashTableConfig htConfig, FragmentContext context,
                     RecordBatch incomingBuild, RecordBatch incomingProbe,
                     RecordBatch outgoing, VectorContainer htContainerOrig) {
-    float loadf = htConfig.getLoadFactor(); 
+    float loadf = htConfig.getLoadFactor();
     int initialCap = htConfig.getInitialCapacity();
 
     if (loadf <= 0 || Float.isNaN(loadf)) throw new IllegalArgumentException("Load factor must be a valid number greater than 0");
@@ -345,18 +345,17 @@ public abstract class HashTableTemplate implements HashTable {
 
     threshold = (int) Math.ceil(tableSize * loadf);
 
-    dummyIntField = MaterializedField.create(new SchemaPath("dummy", ExpressionPosition.UNKNOWN),
-                                             Types.required(MinorType.INT));
+    dummyIntField = MaterializedField.create(SchemaPath.getSimplePath("dummy"), Types.required(MinorType.INT));
 
     startIndices = allocMetadataVector(tableSize, EMPTY_SLOT);
 
-    // Create the first batch holder 
+    // Create the first batch holder
     batchHolders = new ArrayList<BatchHolder>();
     addBatchHolder();
 
     doSetup(incomingBuild, incomingProbe);
 
-    currentIdxHolder = new IntHolder();    
+    currentIdxHolder = new IntHolder();
   }
 
   public int numBuckets() {
@@ -399,7 +398,7 @@ public abstract class HashTableTemplate implements HashTable {
   public PutStatus put(int incomingRowIdx, IntHolder htIdxHolder) {
 
     int hash = getHashBuild(incomingRowIdx);
-    int i = getBucketIndex(hash, numBuckets()); 
+    int i = getBucketIndex(hash, numBuckets());
     int startIdx = startIndices.getAccessor().get(i);
     int currentIdx;
     int currentIdxWithinBatch;
@@ -407,9 +406,9 @@ public abstract class HashTableTemplate implements HashTable {
     BatchHolder lastEntryBatch = null;
     int lastEntryIdxWithinBatch = EMPTY_SLOT;
 
-        
+
     if (startIdx == EMPTY_SLOT) {
-      // this is the first entry in this bucket; find the first available slot in the 
+      // this is the first entry in this bucket; find the first available slot in the
       // container of keys and values
       currentIdx = freeIndex++;
       addBatchIfNeeded(currentIdx);
@@ -430,8 +429,8 @@ public abstract class HashTableTemplate implements HashTable {
 
     bh = batchHolders.get( (currentIdx >>> 16) & BATCH_MASK);
     currentIdxHolder.value = currentIdx;
-    
-    // if startIdx is non-empty, follow the hash chain links until we find a matching 
+
+    // if startIdx is non-empty, follow the hash chain links until we find a matching
     // key or reach the end of the chain
     while (true) {
       currentIdxWithinBatch = currentIdxHolder.value & BATCH_MASK;
@@ -439,7 +438,7 @@ public abstract class HashTableTemplate implements HashTable {
       if (bh.isKeyMatch(incomingRowIdx, currentIdxHolder, false)) {
         htIdxHolder.value = currentIdxHolder.value;
         found = true;
-        break;        
+        break;
       }
       else if (currentIdxHolder.value == EMPTY_SLOT) {
         lastEntryBatch = bh;
@@ -462,7 +461,7 @@ public abstract class HashTableTemplate implements HashTable {
         htIdxHolder.value = currentIdx;
         return PutStatus.KEY_ADDED;
       }
-      else 
+      else
         return PutStatus.PUT_FAILED;
     }
 
@@ -471,7 +470,7 @@ public abstract class HashTableTemplate implements HashTable {
 
   private boolean insertEntry(int incomingRowIdx, int currentIdx, int hashValue, BatchHolder lastEntryBatch, int lastEntryIdx) {
 
-    // resize hash table if needed and transfer the metadata 
+    // resize hash table if needed and transfer the metadata
     resizeAndRehashIfNeeded(currentIdx);
 
     addBatchIfNeeded(currentIdx);
@@ -495,7 +494,7 @@ public abstract class HashTableTemplate implements HashTable {
 
     if (currentIdx == EMPTY_SLOT)
       return -1;
-    
+
     BatchHolder bh = batchHolders.get( (currentIdx >>> 16) & BATCH_MASK);
     currentIdxHolder.value = currentIdx;
 
@@ -503,7 +502,7 @@ public abstract class HashTableTemplate implements HashTable {
 
     while (true) {
       if (bh.isKeyMatch(incomingRowIdx, currentIdxHolder, isProbe)) {
-        found = true; 
+        found = true;
         break;
       } else if (currentIdxHolder.value == EMPTY_SLOT) {
         break;
@@ -511,18 +510,18 @@ public abstract class HashTableTemplate implements HashTable {
         bh = batchHolders.get( (currentIdxHolder.value >>> 16) & BATCH_MASK);
       }
     }
-   
+
     return found ? currentIdxHolder.value : -1;
   }
 
-  // Add a new BatchHolder to the list of batch holders if needed. This is based on the supplied 
+  // Add a new BatchHolder to the list of batch holders if needed. This is based on the supplied
   // currentIdx; since each BatchHolder can hold up to BATCH_SIZE entries, if the currentIdx exceeds
-  // the capacity, we will add a new BatchHolder. 
+  // the capacity, we will add a new BatchHolder.
   private BatchHolder addBatchIfNeeded(int currentIdx) {
     int totalBatchSize = batchHolders.size() * BATCH_SIZE;
-    
+
     if (currentIdx >= totalBatchSize) {
-      BatchHolder bh = addBatchHolder(); 
+      BatchHolder bh = addBatchHolder();
       if (EXTRA_DEBUG) logger.debug("HashTable: Added new batch. Num batches = {}.", batchHolders.size());
       return bh;
     }
@@ -538,19 +537,19 @@ public abstract class HashTableTemplate implements HashTable {
     return bh;
   }
 
-  // Resize the hash table if needed by creating a new one with double the number of buckets. 
+  // Resize the hash table if needed by creating a new one with double the number of buckets.
   // For each entry in the old hash table, re-hash it to the new table and update the metadata
-  // in the new table.. the metadata consists of the startIndices, links and hashValues. 
-  // Note that the keys stored in the BatchHolders are not moved around. 
+  // in the new table.. the metadata consists of the startIndices, links and hashValues.
+  // Note that the keys stored in the BatchHolders are not moved around.
   private void resizeAndRehashIfNeeded(int currentIdx) {
     if (numEntries < threshold)
       return;
 
     if (EXTRA_DEBUG) logger.debug("Hash table numEntries = {}, threshold = {}; resizing the table...", numEntries, threshold);
 
-    // If the table size is already MAXIMUM_CAPACITY, don't resize 
-    // the table, but set the threshold to Integer.MAX_VALUE such that 
-    // future attempts to resize will return immediately. 
+    // If the table size is already MAXIMUM_CAPACITY, don't resize
+    // the table, but set the threshold to Integer.MAX_VALUE such that
+    // future attempts to resize will return immediately.
     if (tableSize == MAXIMUM_CAPACITY) {
       threshold = Integer.MAX_VALUE;
       return;
@@ -570,9 +569,9 @@ public abstract class HashTableTemplate implements HashTable {
     for (int i = 0; i < batchHolders.size(); i++) {
       BatchHolder bh = batchHolders.get(i) ;
       int batchStartIdx = i * BATCH_SIZE;
-      bh.rehash(tableSize, newStartIndices, batchStartIdx);  
-    }    
-    
+      bh.rehash(tableSize, newStartIndices, batchStartIdx);
+    }
+
     startIndices.clear();
     startIndices = newStartIndices;
 
@@ -607,11 +606,11 @@ public abstract class HashTableTemplate implements HashTable {
     return vector;
   }
 
-  // These methods will be code-generated in the context of the outer class 
+  // These methods will be code-generated in the context of the outer class
   protected abstract void doSetup(@Named("incomingBuild") RecordBatch incomingBuild, @Named("incomingProbe") RecordBatch incomingProbe);
   protected abstract int getHashBuild(@Named("incomingRowIdx") int incomingRowIdx) ;
   protected abstract int getHashProbe(@Named("incomingRowIdx") int incomingRowIdx) ;
 
-} 
+}
 
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
index 5fc3733..36428ce 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/orderedpartitioner/OrderedPartitionRecordBatch.java
@@ -84,9 +84,9 @@ import com.sun.codemodel.JExpr;
 public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPartitionSender> {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(OrderedPartitionRecordBatch.class);
 
-  public final MappingSet mainMapping = new MappingSet( (String) null, null, ClassGenerator.DEFAULT_CONSTANT_MAP, 
+  public final MappingSet mainMapping = new MappingSet( (String) null, null, ClassGenerator.DEFAULT_CONSTANT_MAP,
       ClassGenerator.DEFAULT_SCALAR_MAP);
-  public final MappingSet incomingMapping = new MappingSet("inIndex", null, "incoming", null, 
+  public final MappingSet incomingMapping = new MappingSet("inIndex", null, "incoming", null,
       ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
   public final MappingSet partitionMapping = new MappingSet("partitionIndex", null, "partitionVectors", null,
       ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
@@ -131,14 +131,14 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
 
     this.mapKey = String.format("%s_%d", context.getHandle().getQueryId(), context.getHandle().getMajorFragmentId());
     this.minorFragmentSampleCount = cache.getCounter(mapKey);
-    
-    SchemaPath outputPath = new SchemaPath(popConfig.getRef().getPath(), ExpressionPosition.UNKNOWN);
+
+    SchemaPath outputPath = popConfig.getRef();
     MaterializedField outputField = MaterializedField.create(outputPath, Types.required(TypeProtos.MinorType.INT));
     this.partitionKeyVector = (IntVector) TypeHelper.getNewVector(outputField, context.getAllocator());
-    
+
   }
 
-  
+
   @Override
   public void cleanup() {
     super.cleanup();
@@ -152,10 +152,10 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     IterOutcome upstream;
 
     // Start collecting batches until recordsToSample records have been collected
-    
+
     SortRecordBatchBuilder builder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
     builder.add(incoming);
-    
+
     recordsSampled += incoming.getRecordCount();
 
     outer: while (recordsSampled < recordsToSample) {
@@ -211,8 +211,8 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     containerToCache.clear();
     sampleToSave.clear();
     return true;
-    
-    
+
+
   }
 
   /**
@@ -221,7 +221,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
    * distributed cache. Once a sufficient fraction of the fragments have shared their samples, each fragment grabs all
    * the samples, sorts all the records, builds a partition table, and attempts to push the partition table to the
    * distributed cache. Whichever table gets pushed first becomes the table used by all fragments for partitioning.
-   * 
+   *
    * @return True is successful. False if failed.
    */
   private boolean getPartitionVectors() {
@@ -232,7 +232,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
       if (!saveSamples()){
         return false;
       }
-      
+
       VectorAccessibleSerializable finalTable = null;
 
       long val = minorFragmentSampleCount.incrementAndGet();
@@ -282,7 +282,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   private void buildTable() throws SchemaChangeException, ClassTransformationException, IOException {
 
     // Get all samples from distributed map
-    
+
     SortRecordBatchBuilder containerBuilder = new SortRecordBatchBuilder(context.getAllocator(), MAX_SORT_BYTES);
     for (VectorAccessibleSerializable w : mmap.get(mapKey)) {
       containerBuilder.add(w.get());
@@ -293,7 +293,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     List<Ordering> orderDefs = Lists.newArrayList();
     int i = 0;
     for (Ordering od : popConfig.getOrderings()) {
-      SchemaPath sp = new SchemaPath("f" + i++, ExpressionPosition.UNKNOWN);
+      SchemaPath sp = SchemaPath.getSimplePath("f" + i++);
       orderDefs.add(new Ordering(od.getDirection(), new FieldReference(sp)));
     }
 
@@ -317,7 +317,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     WritableBatch batch = WritableBatch.getBatchNoHVWrap(candidatePartitionTable.getRecordCount(), candidatePartitionTable, false);
     VectorAccessibleSerializable wrap = new VectorAccessibleSerializable(batch, context.getDrillbitContext().getAllocator());
     tableMap.putIfAbsent(mapKey + "final", wrap, 1, TimeUnit.MINUTES);
-    
+
     candidatePartitionTable.clear();
     allSamplesContainer.clear();
     containerBuilder.clear();
@@ -330,7 +330,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
    * outgoing. Each Ordering in orderings generates a column, and evaluation of the expression associated with each
    * Ordering determines the value of each column. These records will later be sorted based on the values in each
    * column, in the same order as the orderings.
-   * 
+   *
    * @param sv4
    * @param incoming
    * @param outgoing
@@ -348,7 +348,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     int i = 0;
     for (Ordering od : orderings) {
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), incoming, collector, context.getFunctionRegistry());
-      SchemaPath schemaPath = new SchemaPath("f" + i++, ExpressionPosition.UNKNOWN);
+      SchemaPath schemaPath = SchemaPath.getSimplePath("f" + i++);
       TypeProtos.MajorType.Builder builder = TypeProtos.MajorType.newBuilder().mergeFrom(expr.getMajorType())
           .clearMode().setMode(TypeProtos.DataMode.REQUIRED);
       TypeProtos.MajorType newType = builder.build();
@@ -423,9 +423,9 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
     if (this.first && upstream == IterOutcome.OK_NEW_SCHEMA) {
       if (!getPartitionVectors()){
         cleanup();
-        return IterOutcome.STOP;   
+        return IterOutcome.STOP;
       }
-      
+
       batchQueue = new LinkedBlockingQueue<>(this.sampledIncomingBatches);
       first = false;
 
@@ -497,7 +497,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
   /**
    * Sets up projection that will transfer all of the columns in batch, and also populate the partition column based on
    * which partition a record falls into in the partition table
-   * 
+   *
    * @param batch
    * @throws SchemaChangeException
    */
@@ -547,7 +547,7 @@ public class OrderedPartitionRecordBatch extends AbstractRecordBatch<OrderedPart
 
     try {
       this.projector = context.getImplementationClass(cg);
-      projector.setup(context, batch, this, transfers, partitionVectors, partitions, new SchemaPath(popConfig.getRef().getPath(), ExpressionPosition.UNKNOWN));
+      projector.setup(context, batch, this, transfers, partitionVectors, partitions, popConfig.getRef());
     } catch (ClassTransformationException | IOException e) {
       throw new SchemaChangeException("Failure while attempting to load generated class", e);
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
index cd7e632..94fd385 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/partitionsender/OutgoingRecordBatch.java
@@ -98,7 +98,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
 
   /**
    * Send the record batch to the target node, then reset the value vectors
-   * 
+   *
    * @return true if a flush was needed; otherwise false
    * @throws SchemaChangeException
    */
@@ -106,7 +106,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
     final ExecProtos.FragmentHandle handle = context.getHandle();
 
     if (recordCount != 0) {
-      
+
       for(VectorWrapper<?> w : vectorContainer){
         w.getValueVector().getMutator().setValueCount(recordCount);
       }
@@ -147,7 +147,7 @@ public class OutgoingRecordBatch implements VectorAccessible {
     recordCount = 0;
     vectorContainer.zeroVectors();
     for (VectorWrapper<?> v : vectorContainer) {
-      logger.debug("Reallocating vv to capacity " + DEFAULT_ALLOC_SIZE + " after flush.");
+//      logger.debug("Reallocating vv to capacity " + DEFAULT_ALLOC_SIZE + " after flush.");
       VectorAllocator.getAllocator(v.getValueVector(), DEFAULT_VARIABLE_WIDTH_SIZE).alloc(DEFAULT_ALLOC_SIZE);
     }
     if (!ok) { throw new SchemaChangeException("Flush ended NOT OK!"); }
@@ -173,10 +173,10 @@ public class OutgoingRecordBatch implements VectorAccessible {
       ValueVector outgoingVector = TypeHelper.getNewVector(v.getField(), context.getAllocator());
       VectorAllocator.getAllocator(outgoingVector, 100).alloc(recordCapacity);
       vectorContainer.add(outgoingVector);
-      logger.debug("Reallocating to cap " + recordCapacity + " because of newly init'd vector : " + v.getValueVector());
+//      logger.debug("Reallocating to cap " + recordCapacity + " because of newly init'd vector : " + v.getValueVector());
     }
     outSchema = bldr.build();
-    logger.debug("Initialized OutgoingRecordBatch.  RecordCount: " + recordCount + ", cap: " + recordCapacity + " Schema: " + outSchema);
+//    logger.debug("Initialized OutgoingRecordBatch.  RecordCount: " + recordCount + ", cap: " + recordCapacity + " Schema: " + outSchema);
   }
 
   /**
@@ -226,11 +226,11 @@ public class OutgoingRecordBatch implements VectorAccessible {
     return WritableBatch.getBatchNoHVWrap(recordCount, this, false);
   }
 
-  
+
   private StatusHandler statusHandler = new StatusHandler();
   private class StatusHandler extends BaseRpcOutcomeListener<GeneralRPCProtos.Ack> {
     RpcException ex;
-    
+
     @Override
     public void success(Ack value, ByteBuf buffer) {
       sendCount.decrement();

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
index 4342f52..aaee8e7 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/project/ProjectRecordBatch.java
@@ -24,10 +24,10 @@ import java.util.Set;
 
 import org.apache.drill.common.expression.ErrorCollector;
 import org.apache.drill.common.expression.ErrorCollectorImpl;
-import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FieldReference;
 import org.apache.drill.common.expression.LogicalExpression;
 import org.apache.drill.common.expression.PathSegment;
+import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.SchemaPath;
 import org.apache.drill.common.logical.data.NamedExpression;
 import org.apache.drill.exec.exception.ClassTransformationException;
@@ -58,11 +58,11 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
 
   private Projector projector;
   private List<ValueVector> allocationVectors;
-  
+
   public ProjectRecordBatch(Project pop, RecordBatch incoming, FragmentContext context){
     super(pop, context, incoming);
   }
-  
+
   @Override
   public int getRecordCount() {
     return incoming.getRecordCount();
@@ -85,32 +85,27 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
   private FieldReference getRef(NamedExpression e){
     FieldReference ref = e.getRef();
     PathSegment seg = ref.getRootSegment();
-    if(seg.isNamed() && "output".contentEquals(seg.getNameSegment().getPath())){
-      return new FieldReference(ref.getPath().toString().subSequence(7, ref.getPath().length()), ref.getPosition());
-    }
+
+//    if(seg.isNamed() && "output".contentEquals(seg.getNameSegment().getPath())){
+//      return new FieldReference(ref.getPath().toString().subSequence(7, ref.getPath().length()), ref.getPosition());
+//    }
     return ref;
   }
-  
+
   private boolean isAnyWildcard(List<NamedExpression> exprs){
     for(NamedExpression e : exprs){
       if(isWildcard(e)) return true;
     }
     return false;
   }
-  
+
   private boolean isWildcard(NamedExpression ex){
-    LogicalExpression expr = ex.getExpr();
-    LogicalExpression ref = ex.getRef();
-    if(expr instanceof SchemaPath && ref instanceof SchemaPath){
-      PathSegment e = ((SchemaPath) expr).getRootSegment();
-      PathSegment n = ((SchemaPath) ref).getRootSegment();
-      if(e.isNamed() && e.getNameSegment().getPath().equals("*") && n.isNamed() && n.getChild() != null && n.getChild().isNamed() && n.getChild().getNameSegment().getPath().equals("*")){
-        return true;
-      }
-    }
-    return false;
+    if( !(ex.getExpr() instanceof SchemaPath)) return false;
+    NameSegment expr = ((SchemaPath)ex.getExpr()).getRootSegment();
+    NameSegment ref = ex.getRef().getRootSegment();
+    return ref.getPath().equals("*") && expr.getPath().equals("*");
   }
-  
+
   @Override
   protected void setupNewSchema() throws SchemaChangeException{
     this.allocationVectors = Lists.newArrayList();
@@ -118,7 +113,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
     final List<NamedExpression> exprs = popConfig.getExprs();
     final ErrorCollector collector = new ErrorCollectorImpl();
     final List<TransferPair> transfers = Lists.newArrayList();
-    
+
     final ClassGenerator<Projector> cg = CodeGenerator.getRoot(Projector.TEMPLATE_DEFINITION, context.getFunctionRegistry());
 
     Set<Integer> transferFieldIds = new HashSet();
@@ -128,7 +123,9 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
     if(isAnyWildcard){
       for(VectorWrapper<?> wrapper : incoming){
         ValueVector vvIn = wrapper.getValueVector();
-        TransferPair tp = wrapper.getValueVector().getTransferPair(new FieldReference(vvIn.getField().getName()));
+        String name = vvIn.getField().getDef().getName(vvIn.getField().getDef().getNameCount() - 1).getName();
+        FieldReference ref = new FieldReference(name);
+        TransferPair tp = wrapper.getValueVector().getTransferPair(ref);
         transfers.add(tp);
         container.add(tp.getTo());
       }
@@ -154,7 +151,7 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
           transfers.add(tp);
           container.add(tp.getTo());
           transferFieldIds.add(vectorRead.getFieldId().getFieldId());
-          logger.debug("Added transfer.");
+//          logger.debug("Added transfer.");
         }else{
           // need to do evaluation.
           ValueVector vector = TypeHelper.getNewVector(outputField, context.getAllocator());
@@ -162,15 +159,15 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
           TypedFieldId fid = container.add(vector);
           ValueVectorWriteExpression write = new ValueVectorWriteExpression(fid, expr);
           cg.addExpr(write);
-          logger.debug("Added eval.");
+//          logger.debug("Added eval.");
         }
     }
 
-      
+
     }
-    
+
     container.buildSchema(incoming.getSchema().getSelectionVectorMode());
-    
+
     try {
       this.projector = context.getImplementationClass(cg.getCodeGenerator());
       projector.setup(context, incoming, this, transfers);
@@ -178,6 +175,6 @@ public class ProjectRecordBatch extends AbstractSingleRecordBatch<Project>{
       throw new SchemaChangeException("Failure while attempting to load generated class", e);
     }
   }
-  
-  
+
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
index 5cdab96..29e629a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortBatch.java
@@ -59,7 +59,7 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
   private final SortRecordBatchBuilder builder;
   private Sorter sorter;
   private BatchSchema schema;
-  
+
   public SortBatch(Sort popConfig, FragmentContext context, RecordBatch incoming) {
     super(popConfig, context);
     this.incoming = incoming;
@@ -87,8 +87,8 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
     return builder.getSv4();
   }
 
-  
-  
+
+
   @Override
   public void cleanup() {
     super.cleanup();
@@ -105,8 +105,8 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
         return IterOutcome.NONE;
       }
     }
-    
-    
+
+
     try{
       outer: while (true) {
         IterOutcome upstream = incoming.next();
@@ -134,19 +134,19 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
           throw new UnsupportedOperationException();
         }
       }
-      
-      if (schema == null){
+
+      if (schema == null || builder.isEmpty()){
         // builder may be null at this point if the first incoming batch is empty
         return IterOutcome.NONE;
       }
-        
+
       builder.build(context, container);
       sorter = createNewSorter();
       sorter.setup(context, getSelectionVector4(), this.container);
       sorter.sort(getSelectionVector4(), this.container);
 
       return IterOutcome.OK_NEW_SCHEMA;
-      
+
     }catch(SchemaChangeException | ClassTransformationException | IOException ex){
       kill();
       logger.error("Failure during query", ex);
@@ -163,19 +163,19 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
     final MappingSet mainMapping = new MappingSet( (String) null, null, ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
     final MappingSet leftMapping = new MappingSet("leftIndex", null, ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
     final MappingSet rightMapping = new MappingSet("rightIndex", null, ClassGenerator.DEFAULT_CONSTANT_MAP, ClassGenerator.DEFAULT_SCALAR_MAP);
-    
+
     return createNewSorter(context, orderings, batch, mainMapping, leftMapping, rightMapping);
   }
-  
+
   public static Sorter createNewSorter(FragmentContext context, List<Ordering> orderings, VectorAccessible batch, MappingSet mainMapping, MappingSet leftMapping, MappingSet rightMapping)
           throws ClassTransformationException, IOException, SchemaChangeException{
     CodeGenerator<Sorter> cg = CodeGenerator.get(Sorter.TEMPLATE_DEFINITION, context.getFunctionRegistry());
     ClassGenerator<Sorter> g = cg.getRoot();
     g.setMappingSet(mainMapping);
-    
+
     for(Ordering od : orderings){
       // first, we rewrite the evaluation stack for each side of the comparison.
-      ErrorCollector collector = new ErrorCollectorImpl(); 
+      ErrorCollector collector = new ErrorCollectorImpl();
       final LogicalExpression expr = ExpressionTreeMaterializer.materialize(od.getExpr(), batch, collector,context.getFunctionRegistry());
       if(collector.hasErrors()) throw new SchemaChangeException("Failure while materializing expression. " + collector.toErrorString());
       g.setMappingSet(leftMapping);
@@ -183,26 +183,26 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
       g.setMappingSet(rightMapping);
       HoldingContainer right = g.addExpr(expr, false);
       g.setMappingSet(mainMapping);
-      
+
       // next we wrap the two comparison sides and add the expression block for the comparison.
       LogicalExpression fh = FunctionGenerationHelper.getComparator(left, right, context.getFunctionRegistry());
       HoldingContainer out = g.addExpr(fh, false);
       JConditional jc = g.getEvalBlock()._if(out.getValue().ne(JExpr.lit(0)));
-      
+
       if(od.getDirection() == Direction.ASCENDING){
         jc._then()._return(out.getValue());
       }else{
         jc._then()._return(out.getValue().minus());
       }
     }
-    
+
     g.getEvalBlock()._return(JExpr.lit(0));
-    
+
     return context.getImplementationClass(cg);
 
 
   }
-  
+
   @Override
   public WritableBatch getWritableBatch() {
     throw new UnsupportedOperationException("A sort batch is not writable.");
@@ -213,7 +213,7 @@ public class SortBatch extends AbstractRecordBatch<Sort> {
     incoming.kill();
   }
 
-  
-  
+
+
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
index 8980fdc..bf9db9a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/sort/SortRecordBatchBuilder.java
@@ -38,7 +38,7 @@ import com.google.common.collect.Lists;
 
 public class SortRecordBatchBuilder {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(SortRecordBatchBuilder.class);
-  
+
   private final ArrayListMultimap<BatchSchema, RecordBatchData> batches = ArrayListMultimap.create();
 
   private int recordCount;
@@ -47,12 +47,12 @@ public class SortRecordBatchBuilder {
   private final long maxBytes;
   private SelectionVector4 sv4;
   final PreAllocator svAllocator;
-  
+
   public SortRecordBatchBuilder(BufferAllocator a, long maxBytes){
     this.maxBytes = maxBytes;
     this.svAllocator = a.getNewPreAllocator();
   }
-  
+
   private long getSize(VectorAccessible batch){
     long bytes = 0;
     for(VectorWrapper<?> v : batch){
@@ -60,11 +60,11 @@ public class SortRecordBatchBuilder {
     }
     return bytes;
   }
-  
+
   /**
-   * Add another record batch to the set of record batches.  
+   * Add another record batch to the set of record batches.
    * @param batch
-   * @return True if the requested add completed successfully.  Returns false in the case that this builder is full and cannot receive additional packages. 
+   * @return True if the requested add completed successfully.  Returns false in the case that this builder is full and cannot receive additional packages.
    * @throws SchemaChangeException
    */
   public boolean add(VectorAccessible batch){
@@ -79,7 +79,7 @@ public class SortRecordBatchBuilder {
     if(batchBytes + runningBytes > maxBytes) return false; // enough data memory.
     if(runningBatches+1 > Character.MAX_VALUE) return false; // allowed in batch.
     if(!svAllocator.preAllocate(batch.getRecordCount()*4)) return false;  // sv allocation available.
-      
+
 
     if (batch.getRecordCount() == 0) return true;
     RecordBatchData bd = new RecordBatchData(batch);
@@ -112,15 +112,21 @@ public class SortRecordBatchBuilder {
     return true;
   }
 
+  public boolean isEmpty(){
+    return batches.isEmpty();
+  }
+
   public void build(FragmentContext context, VectorContainer outputContainer) throws SchemaChangeException{
     outputContainer.clear();
     if(batches.keySet().size() > 1) throw new SchemaChangeException("Sort currently only supports a single schema.");
     if(batches.size() > Character.MAX_VALUE) throw new SchemaChangeException("Sort cannot work on more than %d batches at a time.", (int) Character.MAX_VALUE);
-    assert batches.keySet().size() > 0;
+    if(batches.keys().size() < 1){
+      assert false : "Invalid to have an empty set of batches with no schemas.";
+    }
     sv4 = new SelectionVector4(svAllocator.getAllocation(), recordCount, Character.MAX_VALUE);
     BatchSchema schema = batches.keySet().iterator().next();
     List<RecordBatchData> data = batches.get(schema);
-    
+
     // now we're going to generate the sv4 pointers
     switch(schema.getSelectionVectorMode()){
     case NONE: {
@@ -150,7 +156,7 @@ public class SortRecordBatchBuilder {
     default:
       throw new UnsupportedOperationException();
     }
-    
+
     // next, we'll create lists of each of the vector types.
     ArrayListMultimap<MaterializedField, ValueVector> vectors = ArrayListMultimap.create();
     for(RecordBatchData rbd : batches.values()){
@@ -158,12 +164,12 @@ public class SortRecordBatchBuilder {
         vectors.put(v.getField(), v);
       }
     }
-    
+
     for(MaterializedField f : vectors.keySet()){
       List<ValueVector> v = vectors.get(f);
       outputContainer.addHyperList(v, false);
     }
-    
+
     outputContainer.buildSchema(SelectionVectorMode.FOUR_BYTE);
   }
 
@@ -177,7 +183,7 @@ public class SortRecordBatchBuilder {
     }
     if(sv4 != null) sv4.clear();
   }
-  
+
   public List<VectorContainer> getHeldRecordBatches() {
     ArrayList<VectorContainer> containerList = Lists.newArrayList();
     for (BatchSchema bs : batches.keySet()) {
@@ -190,5 +196,5 @@ public class SortRecordBatchBuilder {
     batches.clear();
     return containerList;
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
index 379fad2..8d3a3e5 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/validate/IteratorValidatorBatchIterator.java
@@ -29,26 +29,30 @@ import org.apache.drill.exec.record.WritableBatch;
 import org.apache.drill.exec.record.selection.SelectionVector2;
 import org.apache.drill.exec.record.selection.SelectionVector4;
 
-public class IteratorValidatorBatchIterator implements RecordBatch{
+public class IteratorValidatorBatchIterator implements RecordBatch {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(IteratorValidatorBatchIterator.class);
 
   private IterOutcome state = IterOutcome.NOT_YET;
   private final RecordBatch incoming;
-  
-  public IteratorValidatorBatchIterator(RecordBatch incoming){
+
+  public IteratorValidatorBatchIterator(RecordBatch incoming) {
     this.incoming = incoming;
   }
-  
-  private void validateReadState(){
-    switch(state){
+
+  private void validateReadState() {
+    switch (state) {
     case OK:
     case OK_NEW_SCHEMA:
       return;
     default:
-      throw new IllegalStateException(String.format("You tried to do a batch data read operation when you were in a state of %s.  You can only do this type of operation when you are in a state of OK or OK_NEW_SCHEMA.", state.name()));
+      throw new IllegalStateException(
+          String
+              .format(
+                  "You tried to do a batch data read operation when you were in a state of %s.  You can only do this type of operation when you are in a state of OK or OK_NEW_SCHEMA.",
+                  state.name()));
     }
   }
-  
+
   @Override
   public Iterator<VectorWrapper<?>> iterator() {
     validateReadState();
@@ -105,10 +109,17 @@ public class IteratorValidatorBatchIterator implements RecordBatch{
   public IterOutcome next() {
     if(state == IterOutcome.NONE ) throw new IllegalStateException("The incoming iterator has previously moved to a state of NONE. You should not be attempting to call next() again.");
     state = incoming.next();
-    
-    if ((state == IterOutcome.OK || state == IterOutcome.OK_NEW_SCHEMA) && incoming.getRecordCount() > MAX_BATCH_SIZE)
-      throw new IllegalStateException (String.format("Incoming batch of %s has size %d, which is beyond the limit of %d",  incoming.getClass().getName(), incoming.getRecordCount(), MAX_BATCH_SIZE)); 
-    
+
+    if(state == IterOutcome.OK || state == IterOutcome.OK_NEW_SCHEMA) {
+      BatchSchema schema = incoming.getSchema();
+      if(schema.getFieldCount() == 0){
+        throw new IllegalStateException ("Incoming batch has an empty schema. This is not allowed.");
+      }
+     if(incoming.getRecordCount() > MAX_BATCH_SIZE){
+       throw new IllegalStateException (String.format("Incoming batch of %s has size %d, which is beyond the limit of %d",  incoming.getClass().getName(), incoming.getRecordCount(), MAX_BATCH_SIZE));
+      }
+    }
+
     return state;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
index 42835fc..8bb3d43 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/physical/impl/xsort/ExternalSortBatch.java
@@ -170,7 +170,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         int count = selector.next();
         if(count > 0){
           long t = w.elapsed(TimeUnit.MICROSECONDS);
-          logger.debug("Took {} us to merge {} records", t, count);
+//          logger.debug("Took {} us to merge {} records", t, count);
           container.setRecordCount(count);
           return IterOutcome.OK;
         }else{
@@ -185,7 +185,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
         Stopwatch watch = new Stopwatch();
         watch.start();
         IterOutcome upstream = incoming.next();
-        logger.debug("Took {} us to get next", watch.elapsed(TimeUnit.MICROSECONDS));
+//        logger.debug("Took {} us to get next", watch.elapsed(TimeUnit.MICROSECONDS));
         switch (upstream) {
         case NONE:
           break outer;
@@ -215,7 +215,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
           Stopwatch w = new Stopwatch();
           w.start();
           sorter.sort(sv2);
-          logger.debug("Took {} us to sort {} records", w.elapsed(TimeUnit.MICROSECONDS), sv2.getCount());
+//          logger.debug("Took {} us to sort {} records", w.elapsed(TimeUnit.MICROSECONDS), sv2.getCount());
           batchGroups.add(new BatchGroup(new RecordBatchData(incoming).getContainer(), sv2));
           batchesSinceLastSpill++;
           if (batchGroups.size() > SPILL_THRESHOLD && batchesSinceLastSpill >= SPILL_BATCH_GROUP_SIZE) {
@@ -223,7 +223,7 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
             batchesSinceLastSpill = 0;
           }
           long t = w.elapsed(TimeUnit.MICROSECONDS);
-          logger.debug("Took {} us to sort {} records", t, count);
+//          logger.debug("Took {} us to sort {} records", t, count);
           break;
         default:
           throw new UnsupportedOperationException();
@@ -343,11 +343,11 @@ public class ExternalSortBatch extends AbstractRecordBatch<ExternalSort> {
       ValueVector[] vectors = new ValueVector[batchGroupList.size() * 2];
       int i = 0;
       for (BatchGroup group : batchGroupList) {
-        vectors[i++] = group.getValueAccessorById(group.getValueVectorId(new SchemaPath(field.getName(),ExpressionPosition.UNKNOWN)).getFieldId(),
+        vectors[i++] = group.getValueAccessorById(group.getValueVectorId(field.getAsSchemaPath()).getFieldId(),
                 field.getValueClass()).getValueVector();
         if (group.hasSecond()) {
           VectorContainer c = group.getSecondContainer();
-          vectors[i++] = c.getValueAccessorById(c.getValueVectorId(new SchemaPath(field.getName(),ExpressionPosition.UNKNOWN)).getFieldId(),
+          vectors[i++] = c.getValueAccessorById(c.getValueVectorId(field.getAsSchemaPath()).getFieldId(),
                   field.getValueClass()).getValueVector();
         } else {
           vectors[i] = vectors[i - 1].getTransferPair().getTo(); //this vector should never be used. Just want to avoid having null elements in the hyper vector

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
index 9d02c44..cf3d188 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/common/DrillProjectRelBase.java
@@ -40,7 +40,7 @@ import com.google.common.collect.Lists;
 /**
  *
  * Base class for logical and physical Project implemented in Drill
- */ 
+ */
 public abstract class DrillProjectRelBase extends ProjectRelBase implements DrillRelNode {
   protected DrillProjectRelBase(Convention convention, RelOptCluster cluster, RelTraitSet traits, RelNode child, List<RexNode> exps,
       RelDataType rowType) {
@@ -61,7 +61,7 @@ public abstract class DrillProjectRelBase extends ProjectRelBase implements Dril
     List<NamedExpression> expressions = Lists.newArrayList();
     for (Pair<RexNode, String> pair : projects()) {
       LogicalExpression expr = DrillOptiq.toDrill(context, getChild(), pair.left);
-      expressions.add(new NamedExpression(expr, new FieldReference("output." + pair.right)));
+      expressions.add(new NamedExpression(expr, new FieldReference(pair.right)));
     }
     return expressions;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
index ae777cb..d19b7a4 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/DrillProjectRel.java
@@ -64,13 +64,13 @@ public class DrillProjectRel extends DrillProjectRelBase implements DrillRel {
     }
     return builder.build();
   }
-  
+
   public static DrillProjectRel convert(Project project, ConversionContext context) throws InvalidRelException{
     RelNode input = context.toRel(project.getInput());
     List<RelDataTypeField> fields = Lists.newArrayList();
     List<RexNode> exps = Lists.newArrayList();
     for(NamedExpression expr : project.getSelections()){
-      fields.add(new RelDataTypeFieldImpl(expr.getRef().getPath().toString(), fields.size(), context.getTypeFactory().createSqlType(SqlTypeName.ANY) ));
+      fields.add(new RelDataTypeFieldImpl(expr.getRef().getRootSegment().getPath(), fields.size(), context.getTypeFactory().createSqlType(SqlTypeName.ANY) ));
       exps.add(context.toRex(expr.getExpr()));
     }
     return new DrillProjectRel(context.getCluster(), context.getLogicalTraits(), input, exps, new RelRecordType(fields));

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/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 e5cc730..1492a28 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
@@ -58,12 +58,12 @@ public class DrillRuleSets {
       // Add support for WHERE style joins.
       PushFilterPastProjectRule.INSTANCE,
       PushFilterPastJoinRule.FILTER_ON_JOIN,
-      PushJoinThroughJoinRule.RIGHT, 
-      PushJoinThroughJoinRule.LEFT, 
+      PushJoinThroughJoinRule.RIGHT,
+      PushJoinThroughJoinRule.LEFT,
       // End supprot for WHERE style joins.
-      
+
       //Add back rules
-      
+
       ExpandConversionRule.INSTANCE,
 //      SwapJoinRule.INSTANCE,
       RemoveDistinctRule.INSTANCE,
@@ -80,7 +80,7 @@ public class DrillRuleSets {
 //      SwapJoinRule.INSTANCE, //
 //      PushJoinThroughJoinRule.RIGHT, //
 //      PushJoinThroughJoinRule.LEFT, //
-//      PushSortPastProjectRule.INSTANCE, //      
+//      PushSortPastProjectRule.INSTANCE, //
 
       ////////////////////////////////
       DrillScanRule.INSTANCE,
@@ -91,10 +91,10 @@ public class DrillRuleSets {
       DrillLimitRule.INSTANCE,
       DrillSortRule.INSTANCE,
       DrillJoinRule.INSTANCE,
-      DrillUnionRule.INSTANCE,      
+      DrillUnionRule.INSTANCE,
       MergeProjectRule.INSTANCE
       ));
-  
+
   public static final RuleSet DRILL_PHYSICAL_MEM = new DrillRuleSet(ImmutableSet.of( //
 //      DrillScanRule.INSTANCE,
 //      DrillFilterRule.INSTANCE,
@@ -115,8 +115,9 @@ public class DrillRuleSets {
       StreamAggPrule.INSTANCE,
       MergeJoinPrule.INSTANCE,
       FilterPrule.INSTANCE,
-      LimitPrule.INSTANCE,
-      PushLimitToTopN.INSTANCE
+      LimitPrule.INSTANCE
+
+//      PushLimitToTopN.INSTANCE
 
 //    ExpandConversionRule.INSTANCE,
 //    SwapJoinRule.INSTANCE,
@@ -135,14 +136,14 @@ public class DrillRuleSets {
 //    SwapJoinRule.INSTANCE, //
 //    PushJoinThroughJoinRule.RIGHT, //
 //    PushJoinThroughJoinRule.LEFT, //
-//    PushSortPastProjectRule.INSTANCE, //      
+//    PushSortPastProjectRule.INSTANCE, //
     ));
-  
+
   public static final RuleSet DRILL_PHYSICAL_DISK = new DrillRuleSet(ImmutableSet.of( //
       ProjectPrule.INSTANCE
-  
+
     ));
-  
+
   private static class DrillRuleSet implements RuleSet{
     final ImmutableSet<RelOptRule> rules;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ExprHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ExprHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ExprHelper.java
index 9456a81..128ba28 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ExprHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/logical/ExprHelper.java
@@ -25,17 +25,17 @@ import org.apache.drill.common.expression.SchemaPath;
 
 public class ExprHelper {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(ExprHelper.class);
-  
+
   private final static String COMPOUND_FAIL_MESSAGE = "The current Optiq based logical plan interpreter does not complicated expressions.  For Order By and Filter";
-  
+
   public static String getAggregateFieldName(FunctionCall c){
     List<LogicalExpression> exprs = c.args;
     if(exprs.size() != 1) throw new UnsupportedOperationException(COMPOUND_FAIL_MESSAGE);
     return getFieldName(exprs.iterator().next());
   }
-  
+
   public static String getFieldName(LogicalExpression e){
-    if(e instanceof SchemaPath) return ((SchemaPath) e).getPath().toString();
+    //if(e instanceof SchemaPath) return ((SchemaPath) e).getPath().toString();
     throw new UnsupportedOperationException(COMPOUND_FAIL_MESSAGE);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
index 8573fb2..b75fb40 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/DrillDistributionTrait.java
@@ -18,6 +18,7 @@
 package org.apache.drill.exec.planner.physical;
 
 import org.eigenbase.rel.RelFieldCollation;
+import org.eigenbase.relopt.RelOptPlanner;
 import org.eigenbase.relopt.RelTrait;
 import org.eigenbase.relopt.RelTraitDef;
 
@@ -30,12 +31,12 @@ public class DrillDistributionTrait implements RelTrait {
   public static DrillDistributionTrait SINGLETON = new DrillDistributionTrait(DistributionType.SINGLETON);
   public static DrillDistributionTrait RANDOM_DISTRIBUTED = new DrillDistributionTrait(DistributionType.RANDOM_DISTRIBUTED);
   public static DrillDistributionTrait ANY = new DrillDistributionTrait(DistributionType.ANY);
-  
+
   public static DrillDistributionTrait DEFAULT = ANY;
-  
-  private DistributionType type;  
+
+  private DistributionType type;
   private final ImmutableList<DistributionField> fields;
-  
+
   private DrillDistributionTrait(DistributionType type) {
     assert (type == DistributionType.SINGLETON || type == DistributionType.RANDOM_DISTRIBUTED || type == DistributionType.ANY
             || type == DistributionType.ROUND_ROBIN_DISTRIBUTED || type == DistributionType.BROADCAST_DISTRIBUTED);
@@ -44,11 +45,15 @@ public class DrillDistributionTrait implements RelTrait {
   }
 
   public DrillDistributionTrait(DistributionType type, ImmutableList<DistributionField> fields) {
-    assert (type == DistributionType.HASH_DISTRIBUTED || type == DistributionType.RANGE_DISTRIBUTED);   
+    assert (type == DistributionType.HASH_DISTRIBUTED || type == DistributionType.RANGE_DISTRIBUTED);
     this.type = type;
     this.fields = fields;
   }
 
+  @Override
+  public void register(RelOptPlanner planner) {
+  }
+
   public boolean subsumes(RelTrait trait) {
 
     if (trait instanceof DrillDistributionTrait) {
@@ -65,19 +70,19 @@ public class DrillDistributionTrait implements RelTrait {
           assert(thisFields.size() > 0 && requiredFields.size() > 0);
 
           // A subset of the required distribution columns can satisfy (subsume) the requirement
-          // e.g: required distribution: {a, b, c} 
+          // e.g: required distribution: {a, b, c}
           // Following can satisfy the requirements: {a}, {b}, {c}, {a, b}, {b, c}, {a, c} or {a, b, c}
           return (requiredFields.containsAll(thisFields));
         }
         else if (requiredDist == DistributionType.RANDOM_DISTRIBUTED) {
-          return true; // hash distribution subsumes random distribution and ANY distribution 
+          return true; // hash distribution subsumes random distribution and ANY distribution
         }
       }
     }
 
     return this.equals(trait);
   }
-  
+
   public RelTraitDef<DrillDistributionTrait> getTraitDef() {
     return DrillDistributionTraitDef.INSTANCE;
   }
@@ -93,7 +98,7 @@ public class DrillDistributionTrait implements RelTrait {
   public int hashCode() {
     return  fields == null ? type.hashCode() : type.hashCode() | fields.hashCode() << 4 ;
   }
-  
+
   public boolean equals(Object obj) {
     if (this == obj) {
       return true;
@@ -110,13 +115,13 @@ public class DrillDistributionTrait implements RelTrait {
     return fields == null ? this.type.toString() : this.type.toString() + "(" + fields + ")";
   }
 
-  
+
   public static class DistributionField {
     /**
      * 0-based index of field being DISTRIBUTED.
      */
     private final int fieldId;
-    
+
     public DistributionField (int fieldId) {
       this.fieldId = fieldId;
     }
@@ -128,18 +133,18 @@ public class DrillDistributionTrait implements RelTrait {
       DistributionField other = (DistributionField) obj;
       return this.fieldId == other.fieldId;
     }
-    
+
     public int hashCode() {
       return this.fieldId;
     }
-    
+
     public int getFieldId() {
       return this.fieldId;
     }
-    
+
     public String toString() {
       return String.format("[$%s]", this.fieldId);
     }
   }
-  
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/ecaa838f/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
index f392a18..0fc3abd 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/physical/FilterPrel.java
@@ -40,19 +40,18 @@ public class FilterPrel extends DrillFilterRelBase implements Prel {
   public RelNode copy(RelTraitSet traitSet, List<RelNode> inputs) {
     return new FilterPrel(getCluster(), traitSet, sole(inputs), getCondition());
   }
-  
+
   @Override
   public PhysicalOperator getPhysicalOperator(PhysicalPlanCreator creator) throws IOException {
     Prel child = (Prel) this.getChild();
-    
+
     PhysicalOperator childPOP = child.getPhysicalOperator(creator);
-    
-    //Currently, Filter only accepts "NONE", SV2, SV4. 
-    
+
+    //Currently, Filter accepts "NONE", SV2, SV4.
+
     Filter p = new Filter(childPOP, getFilterExpression(new DrillParseContext()), 1.0f);
-    creator.addPhysicalOperator(p);
-    
+
     return p;
   }
-  
+
 }


[23/51] [abbrv] git commit: Create CastExpression and traverse visitor pattern while performing implicit cast in ExpressionTreeMaterializer.

Posted by ja...@apache.org.
Create CastExpression and traverse visitor pattern while performing implicit cast in ExpressionTreeMaterializer.


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

Branch: refs/heads/master
Commit: 443e4e9fe012147efc0671f9539052ca2b191aeb
Parents: db0ff63
Author: Mehant Baid <me...@gmail.com>
Authored: Mon Mar 31 21:10:51 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:10 2014 -0700

----------------------------------------------------------------------
 .../exec/expr/ExpressionTreeMaterializer.java   | 23 +++++++-------------
 1 file changed, 8 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/443e4e9f/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
index d65ff78..97b965d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/ExpressionTreeMaterializer.java
@@ -127,34 +127,27 @@ public class ExpressionTreeMaterializer {
       if (matchedFuncHolder!=null) {
         //Compare parm type against arg type. Insert cast on top of arg, whenever necessary.
         for (int i = 0; i < call.args.size(); ++i) {
+          LogicalExpression currentArg = call.args.get(i);
+
           TypeProtos.MajorType parmType = matchedFuncHolder.getParmMajorType(i);
 
           //Case 1: If  1) the argument is NullExpression
           //            2) the parameter of matchedFuncHolder allows null input, or func's null_handling is NULL_IF_NULL (means null and non-null are exchangable).
           //        then replace NullExpression with a TypedNullConstant
-          if (call.args.get(i).equals(NullExpression.INSTANCE) &&
+          if (currentArg.equals(NullExpression.INSTANCE) &&
             ( parmType.getMode().equals(TypeProtos.DataMode.OPTIONAL) ||
               matchedFuncHolder.getNullHandling() == FunctionTemplate.NullHandling.NULL_IF_NULL)) {
             argsWithCast.add(new TypedNullConstant(parmType));
-          } else if (Types.softEquals(parmType, call.args.get(i).getMajorType(), matchedFuncHolder.getNullHandling() ==
+          } else if (Types.softEquals(parmType, currentArg.getMajorType(), matchedFuncHolder.getNullHandling() ==
             FunctionTemplate.NullHandling.NULL_IF_NULL)) {
             //Case 2: argument and parameter matches. Do nothing.
             argsWithCast.add(call.args.get(i));
           } else {
             //Case 3: insert cast if param type is different from arg type.
-            String castFuncName = CastFunctions.getCastFunc(parmType.getMinorType());
-            List<LogicalExpression> castArgs = Lists.newArrayList();
-            castArgs.add(call.args.get(i));  //input_expr
-            FunctionCall castCall = new FunctionCall(castFuncName, castArgs, ExpressionPosition.UNKNOWN);
-            DrillFuncHolder matchedCastFuncHolder = resolver.getBestMatch(
-              registry.getDrillRegistry().getMethods().get(castFuncName), castCall);
-
-            if (matchedCastFuncHolder == null) {
-              logFunctionResolutionError(errorCollector, castCall);
-              return NullExpression.INSTANCE;
-            }
-
-            argsWithCast.add(new DrillFuncHolderExpr(call.getName(), matchedCastFuncHolder, castArgs, ExpressionPosition.UNKNOWN));
+
+            // Create the desired output type and CasExpression and traverse the visitor pattern
+            CastExpression castExpression = new CastExpression(currentArg, parmType, ExpressionPosition.UNKNOWN);
+            argsWithCast.add(castExpression.accept(this, registry));
           }
         }
         return new DrillFuncHolderExpr(call.getName(), matchedFuncHolder, argsWithCast, call.getPosition());


[32/51] [abbrv] git commit: initialize min and max functions for bit

Posted by ja...@apache.org.
initialize min and max functions for bit


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

Branch: refs/heads/master
Commit: 3efba1e281b6e47a9b1f3ad47882ffadc97b4335
Parents: 9d233cb
Author: Steven Phillips <sp...@maprtech.com>
Authored: Wed Apr 2 16:28:42 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:11 2014 -0700

----------------------------------------------------------------------
 .../src/main/codegen/templates/AggrTypeFunctions1.java       | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/3efba1e2/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
index 9abc768..14ca1ce 100644
--- a/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
+++ b/exec/java-exec/src/main/codegen/templates/AggrTypeFunctions1.java
@@ -61,7 +61,9 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 	<#if aggrtype.funcName == "sum" || aggrtype.funcName == "count">
 	  value.value = 0;
 	<#elseif aggrtype.funcName == "min">
-	  <#if type.runningType?starts_with("Int")>
+    <#if type.runningType?starts_with("Bit")>
+        value.value = 1;
+	  <#elseif type.runningType?starts_with("Int")>
 	    value.value = Integer.MAX_VALUE;
 	  <#elseif type.runningType?starts_with("BigInt")>
 	    value.value = Long.MAX_VALUE;
@@ -71,7 +73,9 @@ public static class ${type.inputType}${aggrtype.className} implements DrillAggFu
 		value.value = Double.MAX_VALUE;	    
 	  </#if>
 	<#elseif aggrtype.funcName == "max">
-	  <#if type.runningType?starts_with("Int")>
+    <#if type.runningType?starts_with("Bit")>
+        value.value = 0;
+	  <#elseif type.runningType?starts_with("Int")>
 	    value.value = Integer.MIN_VALUE;
 	  <#elseif type.runningType?starts_with("BigInt")>
 	    value.value = Long.MIN_VALUE;


[16/51] [abbrv] git commit: fix or, query uncomment and add absolute reference.

Posted by ja...@apache.org.
fix or, query uncomment and add absolute reference.


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

Branch: refs/heads/master
Commit: 029c5b8980b3cfa0f82ac4461b405e82b650120f
Parents: 255d528
Author: Jacques Nadeau <ja...@apache.org>
Authored: Mon Mar 31 16:32:07 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Sat Apr 19 18:07:09 2014 -0700

----------------------------------------------------------------------
 .../apache/drill/common/expression/FunctionCallFactory.java    | 3 ++-
 .../java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java  | 2 +-
 exec/java-exec/src/test/resources/queries/tpch/06.sql          | 6 +++---
 exec/java-exec/src/test/resources/queries/tpch/16.sql          | 6 +++---
 4 files changed, 9 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/029c5b89/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
index d1f449c..03be80f 100644
--- a/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
+++ b/common/src/main/java/org/apache/drill/common/expression/FunctionCallFactory.java
@@ -46,7 +46,8 @@ public class FunctionCallFactory {
     opToFuncTable.put("%", "modulo");
     opToFuncTable.put("^", "xor");
 
-    opToFuncTable.put("||", "or");
+    opToFuncTable.put("||", "booleanOr");
+    opToFuncTable.put("or", "booleanOr");
     opToFuncTable.put("and", "booleanAnd");
     opToFuncTable.put("&&", "booleanAnd");
     opToFuncTable.put(">", "greater_than");

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/029c5b89/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
index 802595a..1ea4988 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/BitFunctions.java
@@ -29,7 +29,7 @@ import org.apache.drill.exec.record.RecordBatch;
 
 public class BitFunctions {
   
-  @FunctionTemplate(names = {"or", "||"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  @FunctionTemplate(names = {"booleanOr", "or", "||"}, scope = FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
   public static class BitOr implements DrillSimpleFunc {
 
     @Param BitHolder left;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/029c5b89/exec/java-exec/src/test/resources/queries/tpch/06.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/06.sql b/exec/java-exec/src/test/resources/queries/tpch/06.sql
index f1780b6..6d4fac5 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/06.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/06.sql
@@ -5,8 +5,8 @@ select
 from
   cp.`tpch/lineitem.parquet`
 where
---  l_shipdate >= date '1997-01-01'
---  and l_shipdate < date '1997-01-01' + interval '1' year
---  and
+  l_shipdate >= date '1997-01-01'
+  and l_shipdate < date '1997-01-01' + interval '1' year
+  and
   l_discount between 0.03 - 0.01 and 0.03 + 0.01
   and l_quantity < 24;

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/029c5b89/exec/java-exec/src/test/resources/queries/tpch/16.sql
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/resources/queries/tpch/16.sql b/exec/java-exec/src/test/resources/queries/tpch/16.sql
index 96c5fd4..211f2a0 100644
--- a/exec/java-exec/src/test/resources/queries/tpch/16.sql
+++ b/exec/java-exec/src/test/resources/queries/tpch/16.sql
@@ -14,11 +14,11 @@ where
   and p.p_size in (38, 2, 8, 31, 44, 5, 14, 24)
   and ps.ps_suppkey not in (
     select
-      s_suppkey
+      s.s_suppkey
     from
-      cp.`tpch/supplier.parquet`
+      cp.`tpch/supplier.parquet` s
     where
-      s_comment like '%Customer%Complaints%'
+      s.s_comment like '%Customer%Complaints%'
   )
 group by
   p.p_brand,