You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by jn...@apache.org on 2015/01/27 00:05:00 UTC

[1/2] drill git commit: DRILL-2041 : Throw more meaningful error message when Group BY / ORDER BY / COMPARE array/map or repeated scalar type.

Repository: drill
Updated Branches:
  refs/heads/master 3c6d0ef65 -> 3e3388095


DRILL-2041 : Throw more meaningful error message when Group BY / ORDER BY / COMPARE array/map or repeated scalar type.


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

Branch: refs/heads/master
Commit: 3e3388095a0ef1ce1b91f9322363883cb91aba3c
Parents: 0abe273
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Tue Jan 20 18:34:31 2015 -0800
Committer: Jinfeng Ni <jn...@maprtech.com>
Committed: Mon Jan 26 08:12:38 2015 -0800

----------------------------------------------------------------------
 .../exec/expr/fn/FunctionGenerationHelper.java  | 27 ++++++++++++++++++++
 1 file changed, 27 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/3e338809/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
----------------------------------------------------------------------
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
index e0f7f93..d007d7c 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/FunctionGenerationHelper.java
@@ -23,6 +23,7 @@ import java.util.List;
 import org.apache.drill.common.expression.ExpressionPosition;
 import org.apache.drill.common.expression.FunctionHolderExpression;
 import org.apache.drill.common.expression.LogicalExpression;
+import org.apache.drill.common.types.TypeProtos;
 import org.apache.drill.common.types.TypeProtos.MajorType;
 import org.apache.drill.common.types.TypeProtos.MinorType;
 import org.apache.drill.common.types.Types;
@@ -42,6 +43,10 @@ public class FunctionGenerationHelper {
   public static FunctionHolderExpression getComparator(HoldingContainer left,
     HoldingContainer right,
     FunctionImplementationRegistry registry) {
+    if (! isComparableType(left.getMajorType()) || ! isComparableType(right.getMajorType()) ){
+      throw new UnsupportedOperationException(formatCanNotCompareMsg(left.getMajorType(), right.getMajorType()));
+    }
+
     return getFunctionExpression(COMPARE_TO, Types.required(MinorType.INT), registry, left, right);
   }
 
@@ -80,4 +85,26 @@ public class FunctionGenerationHelper {
     sb.append(mt.getMode().name());
   }
 
+  protected static boolean isComparableType(MajorType type) {
+    if (type.getMinorType() == MinorType.MAP ||
+        type.getMinorType() == MinorType.LIST ||
+        type.getMode() == TypeProtos.DataMode.REPEATED ) {
+      return false;
+    } else {
+      return true;
+    }
+  }
+
+  private static String formatCanNotCompareMsg(MajorType left, MajorType right) {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Map, Array or repeated scalar type should not be used in group by, order by or in a comparison operator. Drill does not support compare between ");
+
+    appendType(left, sb);
+    sb.append(" and ");
+    appendType(right, sb);
+    sb.append(".");
+
+    return sb.toString();
+  }
+
 }


[2/2] drill git commit: DRILL-1655: Fix CanNotPlan issue when join Mongodb and other datasource, by ensuring Mongodb plugin return one single instance for each table reference.

Posted by jn...@apache.org.
DRILL-1655:  Fix CanNotPlan issue when join Mongodb and other datasource, by ensuring Mongodb plugin return one single instance for each table reference.


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

Branch: refs/heads/master
Commit: 0abe27387bb7bbcbabd140f569a54c17cd353df0
Parents: 3c6d0ef
Author: Jinfeng Ni <jn...@maprtech.com>
Authored: Fri Jan 23 18:22:16 2015 -0800
Committer: Jinfeng Ni <jn...@maprtech.com>
Committed: Mon Jan 26 08:12:38 2015 -0800

----------------------------------------------------------------------
 .../store/mongo/schema/MongoDatabaseSchema.java | 23 ++++++++++++++++----
 .../store/mongo/schema/MongoSchemaFactory.java  | 15 +++++++++++--
 2 files changed, 32 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/drill/blob/0abe2738/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoDatabaseSchema.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoDatabaseSchema.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoDatabaseSchema.java
index 298d1a9..5e63d81 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoDatabaseSchema.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoDatabaseSchema.java
@@ -18,10 +18,14 @@
 package org.apache.drill.exec.store.mongo.schema;
 
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
 
+import com.google.common.collect.Maps;
 import net.hydromatic.optiq.Table;
 
+import org.apache.drill.exec.planner.logical.DrillTable;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.mongo.MongoStoragePluginConfig;
 import org.apache.drill.exec.store.mongo.schema.MongoSchemaFactory.MongoSchema;
@@ -32,23 +36,34 @@ public class MongoDatabaseSchema extends AbstractSchema {
   static final org.slf4j.Logger logger = org.slf4j.LoggerFactory
       .getLogger(MongoDatabaseSchema.class);
   private final MongoSchema mongoSchema;
-  private final Set<String> tables;
+  private final Set<String> tableNames;
+
+  private final Map<String, DrillTable> drillTables = Maps.newHashMap();
 
   public MongoDatabaseSchema(List<String> tableList, MongoSchema mongoSchema,
       String name) {
     super(mongoSchema.getSchemaPath(), name);
     this.mongoSchema = mongoSchema;
-    this.tables = Sets.newHashSet(tableList);
+    this.tableNames = Sets.newHashSet(tableList);
   }
 
   @Override
   public Table getTable(String tableName) {
-    return mongoSchema.getDrillTable(this.name, tableName);
+    if (!tableNames.contains(tableName)) { // table does not exist
+      return null;
+    }
+
+    if (! drillTables.containsKey(tableName)) {
+      drillTables.put(tableName, mongoSchema.getDrillTable(this.name, tableName));
+    }
+
+    return drillTables.get(tableName);
+
   }
 
   @Override
   public Set<String> getTableNames() {
-    return tables;
+    return tableNames;
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/drill/blob/0abe2738/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
----------------------------------------------------------------------
diff --git a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
index 4abc372..32c42ba 100644
--- a/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
+++ b/contrib/storage-mongo/src/main/java/org/apache/drill/exec/store/mongo/schema/MongoSchemaFactory.java
@@ -21,10 +21,12 @@ import java.net.UnknownHostException;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
 
+import com.google.common.collect.Maps;
 import net.hydromatic.optiq.Schema;
 import net.hydromatic.optiq.SchemaPlus;
 
@@ -34,6 +36,7 @@ import org.apache.drill.exec.planner.logical.DynamicDrillTable;
 import org.apache.drill.exec.rpc.user.UserSession;
 import org.apache.drill.exec.store.AbstractSchema;
 import org.apache.drill.exec.store.SchemaFactory;
+import org.apache.drill.exec.store.dfs.WorkspaceSchemaFactory;
 import org.apache.drill.exec.store.mongo.MongoCnxnManager;
 import org.apache.drill.exec.store.mongo.MongoScanSpec;
 import org.apache.drill.exec.store.mongo.MongoStoragePlugin;
@@ -126,6 +129,8 @@ public class MongoSchemaFactory implements SchemaFactory {
 
   class MongoSchema extends AbstractSchema {
 
+    private final Map<String, MongoDatabaseSchema> schemaMap = Maps.newHashMap();
+
     public MongoSchema(String name) {
       super(ImmutableList.<String> of(), name);
     }
@@ -134,8 +139,14 @@ public class MongoSchemaFactory implements SchemaFactory {
     public Schema getSubSchema(String name) {
       List<String> tables;
       try {
-        tables = tableNameLoader.get(name);
-        return new MongoDatabaseSchema(tables, this, name);
+        if (! schemaMap.containsKey(name)) {
+          tables = tableNameLoader.get(name);
+          schemaMap.put(name, new MongoDatabaseSchema(tables, this, name));
+        }
+
+        return schemaMap.get(name);
+
+        //return new MongoDatabaseSchema(tables, this, name);
       } catch (ExecutionException e) {
         logger.warn("Failure while attempting to access MongoDataBase '{}'.",
             name, e.getCause());