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/20 22:24:59 UTC

[23/32] git commit: DRILL-1015: Move compound identifier converter step into Sql parser.

DRILL-1015: Move compound identifier converter step into Sql parser.

Currently compound identifiers are converted in DrillSqlWorker after
parsing the query, but when views are expanded, we don't apply the
conversion process as the view expander code is in Optiq. Fix is to
move the conversion process to Sql parser it self, so that whenever
a query string is parsed output SqlNode will have compound
identifiers converted.


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

Branch: refs/heads/master
Commit: 136614fdc9bac95d48590cb39ec531da345b88fb
Parents: 2060506
Author: vkorukanti <ve...@gmail.com>
Authored: Thu Jun 19 18:36:27 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Fri Jun 20 10:56:16 2014 -0700

----------------------------------------------------------------------
 .../drill/exec/planner/sql/DrillSqlWorker.java  |  8 ++-
 .../DrillParserWithCompoundIdConverter.java     | 53 ++++++++++++++++++++
 exec/jdbc/pom.xml                               |  1 +
 .../org/apache/drill/jdbc/test/TestViews.java   | 16 ++++++
 exec/jdbc/src/test/resources/nation/nation.tbl  | 25 +++++++++
 5 files changed, 98 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/136614fd/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 c8d2548..cc779ad 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,9 +38,8 @@ 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.planner.sql.parser.impl.DrillParserWithCompoundIdConverter;
 import org.apache.drill.exec.store.StoragePluginRegistry;
 import org.apache.drill.exec.util.Pointer;
 import org.eigenbase.rel.RelCollationTraitDef;
@@ -72,7 +71,7 @@ public class DrillSqlWorker {
         null : new DrillCostBase.DrillCostFactory() ;
     StdFrameworkConfig config = StdFrameworkConfig.newBuilder() //
         .lex(Lex.MYSQL) //
-        .parserFactory(DrillParserImpl.FACTORY) //
+        .parserFactory(DrillParserWithCompoundIdConverter.FACTORY) //
         .defaultSchema(context.getNewDefaultSchema()) //
         .operatorTable(table) //
         .traitDefs(traitDefs) //
@@ -105,8 +104,7 @@ public class DrillSqlWorker {
   }
 
   public PhysicalPlan getPlan(String sql, Pointer<String> textPlan) throws SqlParseException, ValidationException, RelConversionException, IOException{
-    SqlNode originalNode = planner.parse(sql);
-    SqlNode sqlNode = originalNode.accept(new CompoundIdentifierConverter());
+    SqlNode sqlNode = planner.parse(sql);
 
     AbstractSqlHandler handler;
 

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/136614fd/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.java
new file mode 100644
index 0000000..4886741
--- /dev/null
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/planner/sql/parser/impl/DrillParserWithCompoundIdConverter.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.sql.parser.impl;
+
+import org.apache.drill.exec.planner.sql.parser.CompoundIdentifierConverter;
+import org.eigenbase.sql.SqlNode;
+import org.eigenbase.sql.parser.SqlAbstractParserImpl;
+import org.eigenbase.sql.parser.SqlParserImplFactory;
+
+import java.io.Reader;
+
+public class DrillParserWithCompoundIdConverter extends DrillParserImpl {
+
+  /**
+   * {@link org.eigenbase.sql.parser.SqlParserImplFactory} implementation for creating parser.
+   */
+  public static final SqlParserImplFactory FACTORY = new SqlParserImplFactory() {
+    public SqlAbstractParserImpl getParser(Reader stream) {
+      return new DrillParserWithCompoundIdConverter(stream);
+    }
+  };
+
+  public DrillParserWithCompoundIdConverter(Reader stream) {
+    super(stream);
+  }
+
+  @Override
+  public SqlNode parseSqlExpressionEof() throws Exception {
+    SqlNode originalSqlNode = super.parseSqlExpressionEof();
+    return originalSqlNode.accept(new CompoundIdentifierConverter());
+  }
+
+  @Override
+  public SqlNode parseSqlStmtEof() throws Exception {
+    SqlNode originalSqlNode = super.parseSqlStmtEof();
+    return originalSqlNode.accept(new CompoundIdentifierConverter());
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/136614fd/exec/jdbc/pom.xml
----------------------------------------------------------------------
diff --git a/exec/jdbc/pom.xml b/exec/jdbc/pom.xml
index f19294f..1cb5844 100644
--- a/exec/jdbc/pom.xml
+++ b/exec/jdbc/pom.xml
@@ -103,6 +103,7 @@
             <exclude>**/*.json</exclude>
             <exclude>**/git.properties</exclude>
             <exclude>**/donuts-output-data.txt</exclude>
+            <exclude>**/*.tbl</exclude>
             <exclude>**/derby.log</exclude>
           </excludes>
         </configuration>

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/136614fd/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
index 8e7131c..e3f6a8e 100644
--- a/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
+++ b/exec/jdbc/src/test/java/org/apache/drill/jdbc/test/TestViews.java
@@ -219,6 +219,22 @@ public class TestViews extends JdbcTestQueryBase {
   }
 
   @Test
+  public void testViewWithCompoundIdentifiersInSchema() throws Exception{
+    String query = String.format("CREATE VIEW nationview AS SELECT " +
+        "cast(columns[0] AS int) n_nationkey, " +
+        "cast(columns[1] AS CHAR(25)) n_name, " +
+        "cast(columns[2] AS INT) n_regionkey, " +
+        "cast(columns[3] AS VARCHAR(152)) n_comment " +
+        "FROM dfs.`%s/src/test/resources/nation`", WORKING_PATH);
+
+    testViewHelper(
+        query,
+        "nationview",
+        "SELECT * FROM nationview LIMIT 1",
+        "n_nationkey=0; n_name=ALGERIA; n_regionkey=0; n_comment= haggle. carefully final deposits detect slyly agai");
+  }
+
+  @Test
   public void testDropView() throws Exception{
     JdbcAssert.withNoDefaultSchema().withConnection(new Function<Connection, Void>() {
       public Void apply(Connection connection) {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/136614fd/exec/jdbc/src/test/resources/nation/nation.tbl
----------------------------------------------------------------------
diff --git a/exec/jdbc/src/test/resources/nation/nation.tbl b/exec/jdbc/src/test/resources/nation/nation.tbl
new file mode 100644
index 0000000..ed3fd5b
--- /dev/null
+++ b/exec/jdbc/src/test/resources/nation/nation.tbl
@@ -0,0 +1,25 @@
+0|ALGERIA|0| haggle. carefully final deposits detect slyly agai|
+1|ARGENTINA|1|al foxes promise slyly according to the regular accounts. bold requests alon|
+2|BRAZIL|1|y alongside of the pending deposits. carefully special packages are about the ironic forges. slyly special |
+3|CANADA|1|eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold|
+4|EGYPT|4|y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d|
+5|ETHIOPIA|0|ven packages wake quickly. regu|
+6|FRANCE|3|refully final requests. regular, ironi|
+7|GERMANY|3|l platelets. regular accounts x-ray: unusual, regular acco|
+8|INDIA|2|ss excuses cajole slyly across the packages. deposits print aroun|
+9|INDONESIA|2| slyly express asymptotes. regular deposits haggle slyly. carefully ironic hockey players sleep blithely. carefull|
+10|IRAN|4|efully alongside of the slyly final dependencies. |
+11|IRAQ|4|nic deposits boost atop the quickly final requests? quickly regula|
+12|JAPAN|2|ously. final, express gifts cajole a|
+13|JORDAN|4|ic deposits are blithely about the carefully regular pa|
+14|KENYA|0| pending excuses haggle furiously deposits. pending, express pinto beans wake fluffily past t|
+15|MOROCCO|0|rns. blithely bold courts among the closely regular packages use furiously bold platelets?|
+16|MOZAMBIQUE|0|s. ironic, unusual asymptotes wake blithely r|
+17|PERU|1|platelets. blithely pending dependencies use fluffily across the even pinto beans. carefully silent accoun|
+18|CHINA|2|c dependencies. furiously express notornis sleep slyly regular accounts. ideas sleep. depos|
+19|ROMANIA|3|ular asymptotes are about the furious multipliers. express dependencies nag above the ironically ironic account|
+20|SAUDI ARABIA|4|ts. silent requests haggle. closely express packages sleep across the blithely|
+21|VIETNAM|2|hely enticingly express accounts. even, final |
+22|RUSSIA|3| requests against the platelets use never according to the quickly regular pint|
+23|UNITED KINGDOM|3|eans boost carefully special requests. accounts are. carefull|
+24|UNITED STATES|1|y final packages. slow foxes cajole quickly. quickly silent platelets breach ironic accounts. unusual pinto be|