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/25 02:38:12 UTC

[2/2] git commit: Fix schema path to work with array paths

Fix schema path to work with array paths


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

Branch: refs/heads/master
Commit: 1e0e4dfc96c8e7eb83659784dfbb85cd83d7d243
Parents: eba37b5
Author: Jacques Nadeau <ja...@apache.org>
Authored: Thu Apr 24 17:37:49 2014 -0700
Committer: Jacques Nadeau <ja...@apache.org>
Committed: Thu Apr 24 17:37:49 2014 -0700

----------------------------------------------------------------------
 .../drill/common/expression/SchemaPath.java     | 14 +++++----
 .../drill/exec/planner/logical/DrillOptiq.java  | 29 ++++++++++++-------
 .../drill/exec/store/text/TestTextColumn.java   | 30 ++++++++++++++++++++
 3 files changed, 57 insertions(+), 16 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1e0e4dfc/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 28f060e..df96ffc 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
@@ -27,6 +27,7 @@ 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.ArraySegment;
 import org.apache.drill.common.expression.PathSegment.NameSegment;
 import org.apache.drill.common.expression.parser.ExprLexer;
 import org.apache.drill.common.expression.parser.ExprParser;
@@ -35,14 +36,10 @@ 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.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;
 
 public class SchemaPath extends LogicalExpressionBase {
@@ -98,8 +95,13 @@ public class SchemaPath extends LogicalExpressionBase {
   }
 
   public SchemaPath getChild(String childPath){
-    rootSegment.cloneWithNewChild(new NameSegment(childPath));
-    return new SchemaPath(rootSegment);
+    NameSegment newRoot = rootSegment.cloneWithNewChild(new NameSegment(childPath));
+    return new SchemaPath(newRoot);
+  }
+
+  public SchemaPath getChild(int index){
+    NameSegment newRoot = rootSegment.cloneWithNewChild(new ArraySegment(index));
+    return new SchemaPath(newRoot);
   }
 
   public NameSegment getRootSegment() {

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1e0e4dfc/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 46eed45..f46f012 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
@@ -50,6 +50,7 @@ import org.eigenbase.rex.RexRangeRef;
 import org.eigenbase.rex.RexVisitorImpl;
 import org.eigenbase.sql.SqlSyntax;
 import org.eigenbase.sql.fun.SqlStdOperatorTable;
+import org.eigenbase.sql.type.SqlTypeName;
 import org.eigenbase.util.NlsString;
 
 import com.google.common.collect.Lists;
@@ -59,7 +60,7 @@ import com.google.common.collect.Lists;
  */
 public class DrillOptiq {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(DrillOptiq.class);
-  
+
   /**
    * Converts a tree of {@link RexNode} operators into a scalar expression in Drill syntax.
    */
@@ -71,7 +72,7 @@ public class DrillOptiq {
   private static class RexToDrill extends RexVisitorImpl<LogicalExpression> {
     private final RelNode input;
     private final DrillParseContext context;
-    
+
     RexToDrill(DrillParseContext context, RelNode input) {
       super(true);
       this.context = context;
@@ -84,7 +85,7 @@ public class DrillOptiq {
       final RelDataTypeField field = input.getRowType().getFieldList().get(index);
       return new FieldReference(field.getName());
     }
-    
+
     @Override
     public LogicalExpression visitCall(RexCall call) {
       logger.debug("RexCall {}, {}", call);
@@ -153,13 +154,21 @@ public class DrillOptiq {
           }
           return elseExpression;
         }
-        
+
         if (call.getOperator() == SqlStdOperatorTable.ITEM) {
           SchemaPath left = (SchemaPath) call.getOperands().get(0).accept(this);
           final RexLiteral literal = (RexLiteral) call.getOperands().get(1);
-          return left.getChild(literal.getValue2().toString());
+          switch(literal.getTypeName()){
+          case DECIMAL:
+          case INTEGER:
+            return left.getChild(((BigDecimal)literal.getValue()).intValue());
+          case CHAR:
+            return left.getChild(literal.getValue2().toString());
+          default:
+            // fall through
+          }
         }
-        
+
         // fall through
       default:
         throw new AssertionError("todo: implement syntax " + syntax + "(" + call + ")");
@@ -204,13 +213,13 @@ public class DrillOptiq {
     private LogicalExpression getDrillCastFunctionFromOptiq(RexCall call){
       LogicalExpression arg = call.getOperands().get(0).accept(this);
       MajorType castType = null;
-      
+
       switch(call.getType().getSqlTypeName().getName()){
       case "VARCHAR":
       case "CHAR":
         castType = Types.required(MinorType.VARCHAR).toBuilder().setWidth(call.getType().getPrecision()).build();
         break;
-      
+
       case "INTEGER": castType = Types.required(MinorType.INT); break;
       case "FLOAT": castType = Types.required(MinorType.FLOAT4); break;
       case "DOUBLE": castType = Types.required(MinorType.FLOAT8); break;
@@ -236,12 +245,12 @@ public class DrillOptiq {
               throw new UnsupportedOperationException("Only Decimal types with precision range 0 - 38 is supported");
           }
           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()));
       }
-      
+
       return FunctionCallFactory.createCast(castType, ExpressionPosition.UNKNOWN, arg);
 
     }

http://git-wip-us.apache.org/repos/asf/incubator-drill/blob/1e0e4dfc/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
new file mode 100644
index 0000000..fb729d5
--- /dev/null
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/store/text/TestTextColumn.java
@@ -0,0 +1,30 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.drill.exec.store.text;
+
+import org.apache.drill.BaseTestQuery;
+import org.junit.Test;
+
+public class TestTextColumn extends BaseTestQuery{
+  static final org.slf4j.Logger logger = org.slf4j.LoggerFactory.getLogger(TestTextColumn.class);
+
+  @Test
+  public void testCsvColumnSelection() throws Exception{
+    test("select columns[0] as region_id, columns[1] as country from dfs.`[WORKING_PATH]/src/test/resources/store/text/data/regions.csv`");
+  }
+}