You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by ja...@apache.org on 2014/06/16 23:31:31 UTC

[22/32] git commit: DRILL-932: Add support for bracketless complex expressions in SQL queries.

DRILL-932: Add support for bracketless complex expressions in SQL 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/e9ab7aa6
Tree: http://git-wip-us.apache.org/repos/asf/incubator-drill/tree/e9ab7aa6
Diff: http://git-wip-us.apache.org/repos/asf/incubator-drill/diff/e9ab7aa6

Branch: refs/heads/master
Commit: e9ab7aa6f303fccce22cfb83c44ce3edae526768
Parents: 78bc856
Author: Jacques Nadeau <ja...@apache.org>
Authored: Tue Jun 10 20:24:56 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Mon Jun 16 12:44:16 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/FieldReference.java |  16 +-
 .../drill/hbase/TestHBaseFilterPushDown.java    |   4 +-
 .../drill/hbase/TestHBaseProjectPushDown.java   |   8 +-
 exec/java-exec/src/main/codegen/data/Parser.tdd |   4 +
 .../codegen/includes/compoundIdentifier.ftl     |  45 ++++++
 .../planner/common/DrillProjectRelBase.java     |   8 +-
 .../drill/exec/planner/logical/DrillOptiq.java  |   2 +-
 .../drill/exec/planner/sql/DrillSqlWorker.java  |   4 +-
 .../sql/parser/CompoundIdentifierConverter.java | 154 +++++++++++++++++++
 .../sql/parser/DrillCompoundIdentifier.java     | 144 +++++++++++++++++
 .../exec/planner/sql/parser/SqlCreateTable.java |  16 +-
 .../exec/planner/sql/parser/SqlCreateView.java  |  25 ++-
 .../planner/sql/parser/SqlDescribeTable.java    |  12 +-
 .../exec/planner/sql/parser/SqlDropView.java    |   8 +-
 .../exec/planner/sql/parser/SqlShowFiles.java   |  10 +-
 .../exec/planner/sql/parser/SqlShowSchemas.java |  10 +-
 .../exec/planner/sql/parser/SqlShowTables.java  |  12 +-
 .../exec/planner/sql/parser/SqlUseSchema.java   |  14 +-
 .../exec/sql/TestSqlBracketlessSyntax.java      |  68 ++++++++
 .../apache/drill/exec/sql/TestWithClause.java   |  42 +++++
 .../complex/writer/TestComplexTypeReader.java   |  36 ++---
 21 files changed, 579 insertions(+), 63 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/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 bcc6b52..733e585 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
@@ -60,10 +60,22 @@ public class FieldReference extends SchemaPath {
     checkSimpleString(value);
   }
 
+  public static FieldReference getWithQuotedRef(CharSequence safeString){
+    return new FieldReference(safeString, ExpressionPosition.UNKNOWN, false);
+  }
+
+
   public FieldReference(CharSequence value, ExpressionPosition pos) {
+    this(value, pos, true);
+  }
+
+  public FieldReference(CharSequence value, ExpressionPosition pos, boolean check) {
     super(new NameSegment(value), pos);
-    checkData();
-    checkSimpleString(value);
+    if(check){
+      checkData();
+      checkSimpleString(value);
+    }
+
   }
 
   public FieldReference(String value, ExpressionPosition pos, MajorType dataType) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
index 2071e90..50dddeb 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseFilterPushDown.java
@@ -63,9 +63,9 @@ public class TestHBaseFilterPushDown extends BaseHBaseTest {
     runSQLVerifyCount("SELECT\n"
         + "  *\n"
         + "FROM\n"
-        + "  hbase.`[TABLE_NAME]` tableName\n"
+        + "  hbase.`[TABLE_NAME]` t\n"
         + "WHERE\n"
-        + "  (row_key >= 'b5' OR row_key <= 'a2') AND (f['c1'] >= '1' OR f['c1'] is null)"
+        + "  (row_key >= 'b5' OR row_key <= 'a2') AND (t.f.c1 >= '1' OR t.f.c1 is null)"
         , 4);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseProjectPushDown.java
----------------------------------------------------------------------
diff --git a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseProjectPushDown.java b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseProjectPushDown.java
index 7037b9a..ce6f865 100644
--- a/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseProjectPushDown.java
+++ b/contrib/storage-hbase/src/test/java/org/apache/drill/hbase/TestHBaseProjectPushDown.java
@@ -35,9 +35,9 @@ public class TestHBaseProjectPushDown extends BaseHBaseTest {
   public void testColumnWith1RowPushDown() throws Exception{
     setColumnWidth(6);
     runSQLVerifyCount("SELECT\n"
-        + "f2['c7'] as `f[c7]`\n"
+        + "t.f2.c7 as `t.f2.c7`\n"
         + "FROM\n"
-        + "  hbase.`[TABLE_NAME]` tableName"
+        + "  hbase.`[TABLE_NAME]` t"
         , 1);
   }
 
@@ -45,9 +45,9 @@ public class TestHBaseProjectPushDown extends BaseHBaseTest {
   public void testRowKeyAndColumnPushDown() throws Exception{
     setColumnWidths(new int[] {8, 9, 6, 2, 6});
     runSQLVerifyCount("SELECT\n"
-        + "row_key, f['c1']*31 as `f[c1]*31`, f['c2'] as `f[c2]`, 5 as `5`, 'abc' as `'abc'`\n"
+        + "row_key, t.f.c1*31 as `t.f.c1*31`, t.f.c2 as `t.f.c2`, 5 as `5`, 'abc' as `'abc'`\n"
         + "FROM\n"
-        + "  hbase.`[TABLE_NAME]` tableName"
+        + "  hbase.`[TABLE_NAME]` t"
         , 6);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/codegen/data/Parser.tdd
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/data/Parser.tdd b/exec/java-exec/src/main/codegen/data/Parser.tdd
index d781f32..de7af2a 100644
--- a/exec/java-exec/src/main/codegen/data/Parser.tdd
+++ b/exec/java-exec/src/main/codegen/data/Parser.tdd
@@ -65,4 +65,8 @@
   implementationFiles: [
     "parserImpls.ftl"
   ]
+  
+  includeCompoundIdentifier: false
+  
+  
 }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/codegen/includes/compoundIdentifier.ftl
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/codegen/includes/compoundIdentifier.ftl b/exec/java-exec/src/main/codegen/includes/compoundIdentifier.ftl
new file mode 100644
index 0000000..50d8c20
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/includes/compoundIdentifier.ftl
@@ -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. -->
+/**
+ * Parses a Drill compound identifier.
+ */
+SqlIdentifier CompoundIdentifier() :
+{
+    DrillCompoundIdentifier.Builder builder = DrillCompoundIdentifier.newBuilder();
+    String p;
+    int index;
+}
+{
+    p = Identifier()
+    {
+        builder.addString(p, getPos());
+    }
+    (
+        (
+          <DOT> p = Identifier()
+          {
+              builder.addString(p, getPos());
+          }
+        )
+        |
+        (
+          <LBRACKET>
+          index = UnsignedIntLiteral()
+          <RBRACKET> 
+          {
+              builder.addIndex(index, getPos());
+          }
+        )
+    ) *
+    {
+      return builder.build();
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/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 4629737..14817be 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
@@ -55,14 +55,14 @@ public abstract class DrillProjectRelBase extends ProjectRelBase implements Dril
   @Override
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
     if(PrelUtil.getSettings(getCluster()).useDefaultCosting()) {
-      return super.computeSelfCost(planner).multiplyBy(.1); 
+      return super.computeSelfCost(planner).multiplyBy(.1);
     }
-    
+
     // cost is proportional to the number of rows and number of columns being projected
     double rowCount = RelMetadataQuery.getRowCount(this);
     double cpuCost = DrillCostBase.PROJECT_CPU_COST * getRowType().getFieldCount();
     DrillCostFactory costFactory = (DrillCostFactory)planner.getCostFactory();
-    return costFactory.makeCost(rowCount, cpuCost, 0, 0);    
+    return costFactory.makeCost(rowCount, cpuCost, 0, 0);
   }
 
   private List<Pair<RexNode, String>> projects() {
@@ -73,7 +73,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(pair.right)));
+      expressions.add(new NamedExpression(expr, FieldReference.getWithQuotedRef(pair.right)));
     }
     return expressions;
   }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/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 3576622..dd30378 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
@@ -83,7 +83,7 @@ public class DrillOptiq {
     public LogicalExpression visitInputRef(RexInputRef inputRef) {
       final int index = inputRef.getIndex();
       final RelDataTypeField field = input.getRowType().getFieldList().get(index);
-      return new FieldReference(field.getName());
+      return FieldReference.getWithQuotedRef(field.getName());
     }
 
     @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/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 eb2c891..c8d2548 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
@@ -38,6 +38,7 @@ import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
 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.parser.CompoundIdentifierConverter;
 import org.apache.drill.exec.planner.sql.parser.DrillSqlCall;
 import org.apache.drill.exec.planner.sql.parser.impl.DrillParserImpl;
 import org.apache.drill.exec.store.StoragePluginRegistry;
@@ -104,7 +105,8 @@ public class DrillSqlWorker {
   }
 
   public PhysicalPlan getPlan(String sql, Pointer<String> textPlan) throws SqlParseException, ValidationException, RelConversionException, IOException{
-    SqlNode sqlNode = planner.parse(sql);
+    SqlNode originalNode = planner.parse(sql);
+    SqlNode sqlNode = originalNode.accept(new CompoundIdentifierConverter());
 
     AbstractSqlHandler handler;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.java
new file mode 100644
index 0000000..9a44f9c
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/CompoundIdentifierConverter.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.exec.planner.sql.parser;
+
+import java.util.List;
+import java.util.Map;
+
+import org.eigenbase.sql.SqlCall;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.SqlSelect;
+import org.eigenbase.sql.util.SqlShuttle;
+import org.eigenbase.sql.util.SqlVisitor;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+public class CompoundIdentifierConverter extends SqlShuttle {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(CompoundIdentifierConverter.class);
+
+  private boolean enableComplex = true;
+
+  @Override
+  public SqlNode visit(SqlIdentifier id) {
+    if(id instanceof DrillCompoundIdentifier){
+      if(enableComplex){
+        return ((DrillCompoundIdentifier) id).getAsSqlNode();
+      }else{
+        return ((DrillCompoundIdentifier) id).getAsCompoundIdentifier();
+      }
+
+    }else{
+      return id;
+    }
+  }
+
+  @Override
+  public SqlNode visit(final SqlCall call) {
+    // Handler creates a new copy of 'call' only if one or more operands
+    // change.
+    ArgHandler<SqlNode> argHandler = new ComplexExpressionAware(call);
+    call.getOperator().acceptCall(this, call, false, argHandler);
+    return argHandler.result();
+  }
+
+
+  private class ComplexExpressionAware implements ArgHandler<SqlNode>  {
+    boolean update;
+    SqlNode[] clonedOperands;
+    RewriteType[] rewriteTypes;
+    private final SqlCall call;
+
+    public ComplexExpressionAware(SqlCall call) {
+      this.call = call;
+      this.update = false;
+      final List<SqlNode> operands = call.getOperandList();
+      this.clonedOperands = operands.toArray(new SqlNode[operands.size()]);
+      rewriteTypes = REWRITE_RULES.get(call.getClass());
+    }
+
+    public SqlNode result() {
+      if (update) {
+        return call.getOperator().createCall(
+            call.getFunctionQuantifier(),
+            call.getParserPosition(),
+            clonedOperands);
+      } else {
+        return call;
+      }
+    }
+
+    public SqlNode visitChild(
+        SqlVisitor<SqlNode> visitor,
+        SqlNode expr,
+        int i,
+        SqlNode operand) {
+      if (operand == null) {
+        return null;
+      }
+
+      boolean localEnableComplex = enableComplex;
+      if(rewriteTypes != null){
+        switch(rewriteTypes[i]){
+        case DISABLE:
+          enableComplex = false;
+          break;
+        case ENABLE:
+          enableComplex = true;
+        }
+      }
+      SqlNode newOperand = operand.accept(CompoundIdentifierConverter.this);
+      enableComplex = localEnableComplex;
+      if (newOperand != operand) {
+        update = true;
+      }
+      clonedOperands[i] = newOperand;
+      return newOperand;
+    }
+  }
+
+  static final Map<Class<? extends SqlCall>, RewriteType[]> REWRITE_RULES;
+
+  enum RewriteType {
+    UNCHANGED, DISABLE, ENABLE;
+  }
+
+  static {
+    final RewriteType E =RewriteType.ENABLE;
+    final RewriteType D =RewriteType.DISABLE;
+    final RewriteType U =RewriteType.UNCHANGED;
+
+    Map<Class<? extends SqlCall>, RewriteType[]> rules = Maps.newHashMap();
+
+  //SqlNodeList keywordList,
+  //SqlNodeList selectList,
+  //SqlNode fromClause,
+  //SqlNode whereClause,
+  //SqlNodeList groupBy,
+  //SqlNode having,
+  //SqlNodeList windowDecls,
+  //SqlNodeList orderBy,
+  //SqlNode offset,
+  //SqlNode fetch,
+    rules.put(SqlSelect.class, R(D, E, D, E, E, E, D, E, D, D));
+    rules.put(SqlCreateTable.class, R(D, D, E));
+    rules.put(SqlCreateView.class, R(D, E, E, D));
+    rules.put(SqlDescribeTable.class, R(D, D, E));
+    rules.put(SqlDropView.class, R(D));
+    rules.put(SqlShowFiles.class, R(D));
+    rules.put(SqlShowSchemas.class, R(D, D));
+    rules.put(SqlUseSchema.class, R(D));
+    REWRITE_RULES = ImmutableMap.copyOf(rules);
+  }
+
+  private static RewriteType[] R(RewriteType... types){
+    return types;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
new file mode 100644
index 0000000..d718efa
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/DrillCompoundIdentifier.java
@@ -0,0 +1,144 @@
+/**
+ * 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.parser;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.eigenbase.sql.SqlBasicCall;
+import org.eigenbase.sql.SqlIdentifier;
+import org.eigenbase.sql.SqlLiteral;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.fun.SqlStdOperatorTable;
+import org.eigenbase.sql.parser.SqlParserPos;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
+
+public class DrillCompoundIdentifier extends SqlIdentifier{
+
+  List<IdentifierHolder> ids;
+
+  private static List<String> getNames(List<IdentifierHolder> identifiers){
+    List<String> names = Lists.newArrayListWithCapacity(identifiers.size());
+    for(IdentifierHolder h : identifiers){
+      names.add(h.value);
+    }
+    return names;
+  }
+
+  public DrillCompoundIdentifier(List<IdentifierHolder> identifiers) {
+    super(getNames(identifiers), identifiers.get(0).parserPos);
+    this.ids = identifiers;
+  }
+
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillCompoundIdentifier.class);
+
+  public static Builder newBuilder(){
+    return new Builder();
+  }
+
+  public static class Builder {
+    private List<IdentifierHolder> identifiers = Lists.newArrayList();
+
+    public DrillCompoundIdentifier build(){
+      return new DrillCompoundIdentifier(identifiers);
+    }
+
+    public void addString(String name, SqlParserPos pos){
+      identifiers.add(new IdentifierHolder(name, pos, false));
+    }
+
+    public void addIndex(int index, SqlParserPos pos){
+      identifiers.add(new IdentifierHolder(Integer.toString(index), pos, true));
+    }
+  }
+
+  public SqlNode getAsSqlNode(){
+    if(ids.size() == 1){
+      return new SqlIdentifier(Collections.singletonList(ids.get(0).value), ids.get(0).parserPos);
+    }
+
+    int startIndex;
+    SqlNode node;
+
+    if(ids.get(1).isArray()){
+      // handle everything post zero index as item operator.
+      startIndex = 1;
+      node = new SqlIdentifier( //
+          ImmutableList.of(ids.get(0).value), //
+          null, //
+          ids.get(0).parserPos, //
+          ImmutableList.of(ids.get(0).parserPos));
+    }else{
+      // handle everything post two index as item operator.
+      startIndex = 2;
+      node = new SqlIdentifier( //
+          ImmutableList.of(ids.get(0).value, ids.get(1).value), //
+          null, //
+          ids.get(0).parserPos, //
+          ImmutableList.of(ids.get(0).parserPos, ids.get(1).parserPos));
+
+    }
+    for(int i = startIndex ; i < ids.size(); i++){
+      node = ids.get(i).getNode(node);
+    }
+
+    return node;
+  }
+
+
+  public SqlNode getAsCompoundIdentifier(){
+    List<String> names = Lists.newArrayListWithCapacity(ids.size());
+    List<SqlParserPos> pos = Lists.newArrayListWithCapacity(ids.size());
+    for(int i =0; i < ids.size(); i++){
+      IdentifierHolder holder = ids.get(i);
+      names.add(holder.value);
+      pos.add(holder.parserPos);
+    }
+    return new SqlIdentifier(names, null, pos.get(0), pos);
+  }
+
+  private static class IdentifierHolder{
+    String value;
+    SqlParserPos parserPos;
+    boolean isArray;
+
+    public IdentifierHolder(String value, SqlParserPos parserPos, boolean isArray) {
+      super();
+      this.isArray = isArray;
+      this.value = value;
+      this.parserPos = parserPos;
+    }
+
+    public boolean isArray(){
+      return isArray;
+    }
+
+    public SqlNode getNode(SqlNode node){
+      SqlLiteral literal;
+      if(isArray){
+        literal = SqlLiteral.createExactNumeric(value, parserPos);
+      }else{
+        literal = SqlLiteral.createCharString(value, parserPos);
+      }
+      return new SqlBasicCall(SqlStdOperatorTable.ITEM, new SqlNode[]{ node, literal }, parserPos);
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
index 1c916a7..0f0cf0a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateTable.java
@@ -17,7 +17,8 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.List;
+import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
 
 import net.hydromatic.optiq.tools.Planner;
 
@@ -37,7 +38,11 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 public class SqlCreateTable extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_TABLE", SqlKind.OTHER);
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_TABLE", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlCreateTable(pos, (SqlIdentifier) operands[0], (SqlNodeList) operands[1], operands[2]);
+    }
+  };
 
   private SqlIdentifier tblName;
   private SqlNodeList fieldList;
@@ -57,8 +62,11 @@ public class SqlCreateTable extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    if (fieldList == null) return ImmutableList.of(tblName, fieldList);
-    else return ImmutableList.of(tblName, query, fieldList);
+    List<SqlNode> ops = Lists.newArrayList();
+    ops.add(tblName);
+    ops.add(fieldList);
+    ops.add(query);
+    return ops;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
index f3c8bde..6f798b8 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlCreateView.java
@@ -17,7 +17,8 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.List;
+import com.google.common.collect.Lists;
+import com.google.common.collect.ImmutableList;
 
 import net.hydromatic.optiq.tools.Planner;
 
@@ -34,11 +35,14 @@ import org.eigenbase.sql.SqlSpecialOperator;
 import org.eigenbase.sql.SqlWriter;
 import org.eigenbase.sql.parser.SqlParserPos;
 
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Lists;
+import java.util.List;
 
 public class SqlCreateView extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_VIEW", SqlKind.OTHER);
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("CREATE_VIEW", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlCreateView(pos, (SqlIdentifier) operands[0], (SqlNodeList) operands[1], operands[2], (SqlLiteral) operands[3]);
+    }
+  };
 
   private SqlIdentifier viewName;
   private SqlNodeList fieldList;
@@ -46,6 +50,11 @@ public class SqlCreateView extends DrillSqlCall {
   private boolean replaceView;
 
   public SqlCreateView(SqlParserPos pos, SqlIdentifier viewName, SqlNodeList fieldList,
+      SqlNode query, SqlLiteral replaceView) {
+    this(pos, viewName, fieldList, query, replaceView.booleanValue());
+  }
+
+  public SqlCreateView(SqlParserPos pos, SqlIdentifier viewName, SqlNodeList fieldList,
                        SqlNode query, boolean replaceView) {
     super(pos);
     this.viewName = viewName;
@@ -61,8 +70,12 @@ public class SqlCreateView extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    return ImmutableList.of(viewName, fieldList, query,
-        SqlLiteral.createBoolean(replaceView, SqlParserPos.ZERO));
+    List<SqlNode> ops = Lists.newArrayList();
+    ops.add(viewName);
+    ops.add(fieldList);
+    ops.add(query);
+    ops.add(SqlLiteral.createBoolean(replaceView, SqlParserPos.ZERO));
+    return ops;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
index be804a9..8b022b0 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDescribeTable.java
@@ -17,7 +17,7 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
-import java.util.List;
+import com.google.common.collect.Lists;
 
 import net.hydromatic.optiq.tools.Planner;
 
@@ -45,7 +45,11 @@ public class SqlDescribeTable extends DrillSqlCall {
   private final SqlNode columnQualifier;
 
   public static final SqlSpecialOperator OPERATOR =
-    new SqlSpecialOperator("DESCRIBE_TABLE", SqlKind.OTHER);
+    new SqlSpecialOperator("DESCRIBE_TABLE", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlDescribeTable(pos, (SqlIdentifier) operands[0], (SqlIdentifier) operands[1], operands[2]);
+    }
+  };
 
   public SqlDescribeTable(SqlParserPos pos, SqlIdentifier table, SqlIdentifier column, SqlNode columnQualifier) {
     super(pos);
@@ -63,8 +67,8 @@ public class SqlDescribeTable extends DrillSqlCall {
   public List<SqlNode> getOperandList() {
     List<SqlNode> opList = Lists.newArrayList();
     opList.add(table);
-    if (column != null) opList.add(column);
-    if (columnQualifier != null) opList.add(columnQualifier);
+    opList.add(column);
+    opList.add(columnQualifier);
     return opList;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
index 20a7952..b86bd20 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlDropView.java
@@ -35,7 +35,11 @@ import org.eigenbase.sql.parser.SqlParserPos;
 import com.google.common.collect.ImmutableList;
 
 public class SqlDropView extends DrillSqlCall {
-  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_VIEW", SqlKind.OTHER);
+  public static final SqlSpecialOperator OPERATOR = new SqlSpecialOperator("DROP_VIEW", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlDropView(pos, (SqlIdentifier) operands[0]);
+    }
+  };
 
   private SqlIdentifier viewName;
 
@@ -51,7 +55,7 @@ public class SqlDropView extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    return ImmutableList.of((SqlNode)viewName);
+    return Collections.singletonList((SqlNode)viewName);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowFiles.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowFiles.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowFiles.java
index 7e967b6..c929db6 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowFiles.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowFiles.java
@@ -43,7 +43,11 @@ public class SqlShowFiles extends DrillSqlCall {
   private final SqlIdentifier db;
 
   public static final SqlSpecialOperator OPERATOR =
-      new SqlSpecialOperator("SHOW_FILES", SqlKind.OTHER);
+      new SqlSpecialOperator("SHOW_FILES", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlShowFiles(pos, (SqlIdentifier) operands[0]);
+    }
+  };
 
   public SqlShowFiles(SqlParserPos pos, SqlIdentifier db) {
     super(pos);
@@ -57,9 +61,7 @@ public class SqlShowFiles extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    List<SqlNode> opList = Lists.newArrayList();
-    if (db != null) opList.add(db);
-    return opList;
+    return Collections.singletonList( (SqlNode) db);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowSchemas.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowSchemas.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowSchemas.java
index 2ef6553..03e4e1d 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowSchemas.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowSchemas.java
@@ -43,7 +43,11 @@ public class SqlShowSchemas extends DrillSqlCall {
   private final SqlNode whereClause;
 
   public static final SqlSpecialOperator OPERATOR =
-    new SqlSpecialOperator("SHOW_SCHEMAS", SqlKind.OTHER);
+    new SqlSpecialOperator("SHOW_SCHEMAS", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlShowSchemas(pos, operands[0], operands[1]);
+    }
+  };
 
   public SqlShowSchemas(SqlParserPos pos, SqlNode likePattern, SqlNode whereClause) {
     super(pos);
@@ -59,8 +63,8 @@ public class SqlShowSchemas extends DrillSqlCall {
   @Override
   public List<SqlNode> getOperandList() {
     List<SqlNode> opList = Lists.newArrayList();
-    if (likePattern != null) opList.add(likePattern);
-    if (whereClause != null) opList.add(whereClause);
+    opList.add(likePattern);
+    opList.add(whereClause);
     return opList;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowTables.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowTables.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowTables.java
index 93de127..ae95944 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowTables.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlShowTables.java
@@ -45,7 +45,11 @@ public class SqlShowTables extends DrillSqlCall {
   private final SqlNode whereClause;
 
   public static final SqlSpecialOperator OPERATOR =
-    new SqlSpecialOperator("SHOW_TABLES", SqlKind.OTHER);
+    new SqlSpecialOperator("SHOW_TABLES", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlShowTables(pos, (SqlIdentifier) operands[0], operands[1], operands[2]);
+    }
+  };
 
   public SqlShowTables(SqlParserPos pos, SqlIdentifier db, SqlNode likePattern, SqlNode whereClause) {
     super(pos);
@@ -62,9 +66,9 @@ public class SqlShowTables extends DrillSqlCall {
   @Override
   public List<SqlNode> getOperandList() {
     List<SqlNode> opList = Lists.newArrayList();
-    if (db != null) opList.add(db);
-    if (likePattern != null) opList.add(likePattern);
-    if (whereClause != null) opList.add(whereClause);
+    opList.add(db);
+    opList.add(likePattern);
+    opList.add(whereClause);
     return opList;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
index 554bfbf..76c7df3 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/SqlUseSchema.java
@@ -17,6 +17,7 @@
  */
 package org.apache.drill.exec.planner.sql.parser;
 
+import java.util.Collections;
 import java.util.List;
 
 import net.hydromatic.optiq.tools.Planner;
@@ -24,23 +25,28 @@ import net.hydromatic.optiq.tools.Planner;
 import org.apache.drill.exec.ops.QueryContext;
 import org.apache.drill.exec.planner.sql.handlers.AbstractSqlHandler;
 import org.apache.drill.exec.planner.sql.handlers.UseSchemaHandler;
+import org.eigenbase.sql.SqlCall;
 import org.eigenbase.sql.SqlIdentifier;
 import org.eigenbase.sql.SqlKind;
+import org.eigenbase.sql.SqlLiteral;
 import org.eigenbase.sql.SqlNode;
 import org.eigenbase.sql.SqlOperator;
 import org.eigenbase.sql.SqlSpecialOperator;
 import org.eigenbase.sql.SqlWriter;
 import org.eigenbase.sql.parser.SqlParserPos;
 
-import com.google.common.collect.ImmutableList;
-
 /**
  * Sql parser tree node to represent <code>USE SCHEMA</code> statement.
  */
 public class SqlUseSchema extends DrillSqlCall {
 
   public static final SqlSpecialOperator OPERATOR =
-      new SqlSpecialOperator("USE_SCHEMA", SqlKind.OTHER);
+      new SqlSpecialOperator("USE_SCHEMA", SqlKind.OTHER){
+    public SqlCall createCall(SqlLiteral functionQualifier, SqlParserPos pos, SqlNode... operands) {
+      return new SqlUseSchema(pos, (SqlIdentifier) operands[0]);
+    }
+  };
+
   private SqlIdentifier schema;
 
   public SqlUseSchema(SqlParserPos pos, SqlIdentifier schema) {
@@ -51,7 +57,7 @@ public class SqlUseSchema extends DrillSqlCall {
 
   @Override
   public List<SqlNode> getOperandList() {
-    return ImmutableList.of((SqlNode)schema);
+    return Collections.singletonList((SqlNode)schema);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.java
new file mode 100644
index 0000000..1204e7c
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestSqlBracketlessSyntax.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.sql;
+
+import net.hydromatic.optiq.config.Lex;
+import net.hydromatic.optiq.tools.Frameworks;
+import net.hydromatic.optiq.tools.Planner;
+import net.hydromatic.optiq.tools.StdFrameworkConfig;
+
+import org.apache.drill.exec.planner.sql.DrillConvertletTable;
+import org.apache.drill.exec.planner.sql.parser.CompoundIdentifierConverter;
+import org.apache.drill.exec.planner.sql.parser.impl.DrillParserImpl;
+import org.eigenbase.sql.SqlNode;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class TestSqlBracketlessSyntax {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestSqlBracketlessSyntax.class);
+
+  @Test
+  public void checkComplexExpressionParsing() throws Exception{
+    StdFrameworkConfig config = StdFrameworkConfig.newBuilder() //
+        .lex(Lex.MYSQL) //
+        .parserFactory(DrillParserImpl.FACTORY) //
+        .defaultSchema(Frameworks.createRootSchema(false)) //
+        .convertletTable(new DrillConvertletTable()) //
+        .build();
+    Planner planner = Frameworks.getPlanner(config);
+
+    SqlNode node = planner.parse(""
+        + "select a[4].c \n"
+        + "from x.y.z \n"
+        + "where a.c.b = 5 and x[2] = 7 \n"
+        + "group by d \n"
+        + "having a.c < 5 \n"
+        + "order by x.a.a.a.a.a");
+
+    String expected = "SELECT `a`[4]['c']\n" +
+        "FROM `x`.`y`.`z`\n" +
+        "WHERE `a`.`c`['b'] = 5 AND `x`[2] = 7\n" +
+        "GROUP BY `d`\n" +
+        "HAVING `a`.`c` < 5\n" +
+        "ORDER BY `x`.`a`['a']['a']['a']['a']";
+
+
+    SqlNode rewritten = node.accept(new CompoundIdentifierConverter());
+    String rewrittenQuery = rewritten.toString();
+
+    Assert.assertEquals(expected, rewrittenQuery);
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.java
new file mode 100644
index 0000000..3a32045
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/sql/TestWithClause.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.sql;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Ignore;
+import org.junit.Test;
+
+public class TestWithClause extends BaseTestQuery {
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestWithClause.class);
+
+  @Test
+  public void withClause() throws Exception {
+    test("with alpha as (select * from sys.options where type = 'SYSTEM') \n" +
+        "\n" +
+        "select * from alpha");
+  }
+
+  @Test
+  @Ignore
+  public void withClauseWithAliases() throws Exception {
+    test("with alpha (x,y) as (select name, kind from sys.options where type = 'SYSTEM') \n" +
+        "\n" +
+        "select x, y from alpha");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/e9ab7aa6/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
index 74c724d..3da3543 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/vector/complex/writer/TestComplexTypeReader.java
@@ -25,21 +25,21 @@ public class TestComplexTypeReader extends BaseTestQuery{
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestComplexTypeReader.class);
 
   @Test
-  // Repeated map (map) -> json. 
+  // Repeated map (map) -> json.
   public void testX() throws Exception{
     test("select convert_to(z[0], 'JSON') from cp.`jsoninput/input2.json`;");
   }
 
   @Test
-  //map -> json. 
+  //map -> json.
   public void testX2() throws Exception{
     test("select convert_to(x, 'JSON') from cp.`jsoninput/input2.json`;");
   }
 
   @Test
-  //Map (mapfield) -> json.  
+  //Map (mapfield) -> json.
   public void testX3() throws Exception{
-    test("select convert_to(x['y'], 'JSON') from cp.`jsoninput/input2.json`;");
+    test("select convert_to(tbl.x.y, 'JSON') from cp.`jsoninput/input2.json` tbl;");
   }
 
   @Test
@@ -53,9 +53,9 @@ public class TestComplexTypeReader extends BaseTestQuery{
   public void testX5() throws Exception{
     test("select convert_to(`integer`, 'JSON') from cp.`jsoninput/input2.json`;");
   }
-  
+
   @Test
-  // repeated map -> json. 
+  // repeated map -> json.
   public void testX6() throws Exception{
     test("select convert_to(z, 'JSON')  from cp.`jsoninput/input2.json`;");
   }
@@ -90,7 +90,7 @@ public class TestComplexTypeReader extends BaseTestQuery{
 
   @Test
   public void testY3() throws Exception{
-    test("select x['y'] from cp.`jsoninput/input2.json`;");
+    test("select tbl.x.y from cp.`jsoninput/input2.json` tbl;");
   }
 
   @Test
@@ -99,13 +99,13 @@ public class TestComplexTypeReader extends BaseTestQuery{
   }
 
   @Test
-  //repeated list (Repeated BigInt) 
+  //repeated list (Repeated BigInt)
   public void testZ() throws Exception{
     test("select rl[1] from cp.`jsoninput/input2.json`;");
   }
 
   @Test
-  //repeated list (Repeated BigInt ( BigInt) ) ) 
+  //repeated list (Repeated BigInt ( BigInt) ) )
   public void testZ1() throws Exception{
     test("select rl[0][1] from cp.`jsoninput/input2.json`;");
   }
@@ -123,27 +123,27 @@ public class TestComplexTypeReader extends BaseTestQuery{
   }
 
   @Test
-  //repeated map  --> Json.  It will go beyond the buffer of size 256 allocated in setup. 
+  //repeated map  --> Json.  It will go beyond the buffer of size 256 allocated in setup.
   public void testA0() throws Exception{
     test("  select convert_to(types, 'JSON') from cp.`jsoninput/vvtypes.json`;");
   }
 
   @Test
-  //repeated map (map) --> Json. 
+  //repeated map (map) --> Json.
   public void testA1() throws Exception{
     test("  select convert_to(types[1], 'JSON') from cp.`jsoninput/vvtypes.json`;");
   }
 
   @Test
-  //repeated map (map (repeated map) ) --> Json. 
+  //repeated map (map (repeated map) ) --> Json.
   public void testA2() throws Exception{
-    test("  select convert_to(types[1]['minor'], 'JSON') from cp.`jsoninput/vvtypes.json`;");
+    test("  select convert_to(types[1].minor, 'JSON') from cp.`jsoninput/vvtypes.json`;");
   }
-  
+
   @Test
-  //repeated map (map( repeated map (map (varchar)))) --> Json. 
+  //repeated map (map( repeated map (map (varchar)))) --> Json.
   public void testA3() throws Exception{
-    test("  select convert_to(types[1]['minor'][0]['valueHolder'], 'JSON') from cp.`jsoninput/vvtypes.json`;");
+    test("  select convert_to(types[1].minor[0].valueHolder, 'JSON') from cp.`jsoninput/vvtypes.json`;");
   }
 
   @Test
@@ -155,13 +155,13 @@ public class TestComplexTypeReader extends BaseTestQuery{
   @Test
   //repeated map (map (repeated map) ).
   public void testB2() throws Exception{
-    test("  select types[1]['minor'] from cp.`jsoninput/vvtypes.json`;");
+    test("  select types[1].minor from cp.`jsoninput/vvtypes.json`;");
   }
 
   @Test
   //repeated map (map( repeated map (map (varchar)))).
   public void testB3() throws Exception{
-    test("  select types[1]['minor'][0]['valueHolder'] from cp.`jsoninput/vvtypes.json`;");
+    test("  select types[1].minor[0].valueholder from cp.`jsoninput/vvtypes.json`;");
   }
 
 }