You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by cg...@apache.org on 2021/02/13 23:59:35 UTC

[drill] branch master updated: DRILL-7820: Not Supporting Mongo DB Name Capital Letters

This is an automated email from the ASF dual-hosted git repository.

cgivre pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/drill.git


The following commit(s) were added to refs/heads/master by this push:
     new c3f2a93  DRILL-7820: Not Supporting Mongo DB Name Capital Letters
c3f2a93 is described below

commit c3f2a933e3f2f6893c6054ec9dbe7d067ffa5c70
Author: luocooong <lu...@apache.org>
AuthorDate: Fri Feb 12 09:06:33 2021 +0800

    DRILL-7820: Not Supporting Mongo DB Name Capital Letters
---
 .../store/mongo/schema/MongoSchemaFactory.java     | 24 ++++++++--
 .../drill/exec/store/mongo/MongoTestConstants.java |  6 ++-
 .../drill/exec/store/mongo/MongoTestSuite.java     |  1 +
 .../exec/store/mongo/TestMongoDrillIssue.java      | 52 ++++++++++++++++++++++
 4 files changed, 78 insertions(+), 5 deletions(-)

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 fdba6b0..f5623d8 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
@@ -20,11 +20,13 @@ package org.apache.drill.exec.store.mongo.schema;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.TimeUnit;
+import java.util.function.Consumer;
 
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.drill.common.exceptions.DrillRuntimeException;
@@ -57,11 +59,13 @@ public class MongoSchemaFactory extends AbstractSchemaFactory {
 
   private LoadingCache<String, List<String>> databases;
   private LoadingCache<String, List<String>> tableNameLoader;
+  private Map<String, String> schemaNameMap;
   private final MongoStoragePlugin plugin;
 
   public MongoSchemaFactory(MongoStoragePlugin plugin, String schemaName) throws ExecutionSetupException {
     super(schemaName);
     this.plugin = plugin;
+    this.schemaNameMap = new HashMap<String, String>();
 
     databases = CacheBuilder //
         .newBuilder() //
@@ -83,7 +87,21 @@ public class MongoSchemaFactory extends AbstractSchemaFactory {
       }
       try {
         List<String> dbNames = new ArrayList<>();
-        plugin.getClient().listDatabaseNames().into(dbNames);
+        plugin.getClient().listDatabaseNames().forEach(new Consumer<String>() {
+          @Override
+          public void accept(String name) {
+            // 1. Schemas in drill are case insensitive and stored in lower case.
+            dbNames.add(name.toLowerCase());
+            /**
+             * 2. Support database name with capital letters.
+             * case 1: "show tables from mongo.HELLO", Should using the lower case name
+             *  to resolve the schema lookup in `CalciteSchema`.
+             * case 2: "select * from mongo.HEllO.myTable", Must be using origin name
+             *  to create `MongoScanSpec` and initial connection in `MongoRecordReader`.
+             */
+            schemaNameMap.put(name.toLowerCase(), name);
+          }
+        });
         return dbNames;
       } catch (MongoException me) {
         logger.warn("Failure while loading databases in Mongo. {}",
@@ -101,7 +119,7 @@ public class MongoSchemaFactory extends AbstractSchemaFactory {
     @Override
     public List<String> load(String dbName) throws Exception {
       try {
-        MongoDatabase db = plugin.getClient().getDatabase(dbName);
+        MongoDatabase db = plugin.getClient().getDatabase(schemaNameMap.get(dbName));
         List<String> collectionNames = new ArrayList<>();
         db.listCollectionNames().into(collectionNames);
         return collectionNames;
@@ -183,7 +201,7 @@ public class MongoSchemaFactory extends AbstractSchemaFactory {
     }
 
     DrillTable getDrillTable(String dbName, String collectionName) {
-      MongoScanSpec mongoScanSpec = new MongoScanSpec(dbName, collectionName);
+      MongoScanSpec mongoScanSpec = new MongoScanSpec(schemaNameMap.get(dbName), collectionName);
       return new DynamicDrillTable(plugin, getName(), null, mongoScanSpec);
     }
 
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
index a6d9d2f..d8190a6 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestConstants.java
@@ -37,19 +37,21 @@ public interface MongoTestConstants {
   String EMPLOYEE_DB = "employee";
   String AUTHENTICATION_DB = "admin";
   String DONUTS_DB = "donuts";
+  String DATATYPE_DB = "datatype";
+  String ISSUE7820_DB = "ISSUE7820"; // capital letters
 
   String DONUTS_COLLECTION = "donuts";
   String EMPINFO_COLLECTION = "empinfo";
   String EMPTY_COLLECTION = "empty";
   String SCHEMA_CHANGE_COLLECTION = "schema_change";
+  String DATATYPE_COLLECTION = "types";
+  String ISSUE7820_COLLECTION = "Issue7820";
 
   String DONUTS_DATA = "donuts.json";
   String EMP_DATA = "emp.json";
   String SCHEMA_CHANGE_DATA = "schema_change_int_to_string.json";
 
   String STORAGE_ENGINE = "wiredTiger";
-  String DATATYPE_DB = "datatype";
-  String DATATYPE_COLLECTION = "types";
   String DATATYPE_DATA = "datatype-oid.json";
 
   String CONFIG_REPLICA_SET = "config_replicas";
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
index 7797cdd..680281e 100644
--- a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/MongoTestSuite.java
@@ -258,6 +258,7 @@ public class MongoTestSuite extends BaseTest implements MongoTestConstants {
         TestTableGenerator.importData(EMPLOYEE_DB, SCHEMA_CHANGE_COLLECTION, SCHEMA_CHANGE_DATA);
         TestTableGenerator.importData(DONUTS_DB, DONUTS_COLLECTION, DONUTS_DATA);
         TestTableGenerator.importData(DATATYPE_DB, DATATYPE_COLLECTION, DATATYPE_DATA);
+        TestTableGenerator.importData(ISSUE7820_DB, ISSUE7820_COLLECTION, EMP_DATA);
       }
       initCount.incrementAndGet();
     }
diff --git a/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoDrillIssue.java b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoDrillIssue.java
new file mode 100644
index 0000000..24f574e
--- /dev/null
+++ b/contrib/storage-mongo/src/test/java/org/apache/drill/exec/store/mongo/TestMongoDrillIssue.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.store.mongo;
+
+import org.apache.drill.categories.MongoStorageTest;
+import org.apache.drill.categories.SlowTest;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+
+@Category({SlowTest.class, MongoStorageTest.class})
+public class TestMongoDrillIssue extends MongoTestBase {
+
+  // DRILL-7820: Not support database name in capital letters.
+  @Test
+  public void testIssue7820() throws Exception {
+    testBuilder()
+    .sqlQuery("show tables from mongo.ISSUE7820")
+    .unOrdered()
+    .baselineColumns("TABLE_SCHEMA", "TABLE_NAME")
+    .baselineValues("mongo.issue7820", "Issue7820")
+    .go();
+  }
+
+  // Add more test for DRILL-7820
+  @Test
+  public void testIssue7820Select() throws Exception {
+    testBuilder()
+    .sqlQuery("select employee_id, full_name from mongo.ISSUE7820.Issue7820 limit 5 offset 15")
+    .unOrdered()
+    .baselineColumns("employee_id", "full_name")
+    .baselineValues(1116, "Phil Munoz")
+    .baselineValues(1117, "Lori Lightfoot")
+    .baselineValues(1, "Kumar")
+    .baselineValues(2, "Kamesh")
+    .go();
+  }
+}