You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by cs...@apache.org on 2020/10/07 18:58:27 UTC

[impala] branch master updated: IMPALA-10172: Support Hive metastore managed locations for databases

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 76f9b75  IMPALA-10172: Support Hive metastore managed locations for databases
76f9b75 is described below

commit 76f9b75c8b701e50378ce92a729f01768da55171
Author: Csaba Ringhofer <cs...@cloudera.com>
AuthorDate: Thu Oct 1 12:01:39 2020 +0200

    IMPALA-10172: Support Hive metastore managed locations for databases
    
    This change lets the user set the managed location path in new
    databases, e.g.
    CREATE DATABASE db MANAGEDLOCATION 'some url';
    
    This property sets the location where the database's tables with
    table property 'transactional'='true' will be placed.
    
    The change also adds managedlocation to DESCRIBE DATABASE's output.
    Example:
    DESCRIBE DATABASE db;
    +------------------+-----------------------------------------+---------+
    | name             | location                                | comment |
    +------------------+-----------------------------------------+---------+
    | db               | hdfs://localhost:20500/test-warehouse/a |         |
    | managedlocation: | hdfs://localhost:20500/test-warehouse/b |         |
    +------------------+-----------------------------------------+---------+
    DESCRIBE DATABASE EXTENDED db6;
    +------------------+-----------------------------------------+---------+
    | name             | location                                | comment |
    +------------------+-----------------------------------------+---------+
    | db               | hdfs://localhost:20500/test-warehouse/a |         |
    | managedlocation: | hdfs://localhost:20500/test-warehouse/b |         |
    | Owner:           |                                         |         |
    |                  | csringhofer                             | USER    |
    +------------------+-----------------------------------------+---------+
    
    Note that Impala's output for DESCRIBE DATABASE (EXTENDED) is
    different than Hive's, where a new column was added for each extra
    piece of information, while Impala adds a new row to keep the 3 column
    format. Changing to Hive's format would be preferable in my opinion,
    but is a potentially breaking change.
    See IMPALA-6686 for further discussion.
    
    Testing:
    - added FE and EE tests
    - ran relevant tests
    
    Change-Id: I925632a43ff224f762031e89981896722e453399
    Reviewed-on: http://gerrit.cloudera.org:8080/16529
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 common/thrift/JniCatalog.thrift                    |  4 ++
 fe/src/main/cup/sql-parser.cup                     | 19 +++++-
 .../org/apache/impala/analysis/CreateDbStmt.java   | 14 ++++-
 .../apache/impala/service/CatalogOpExecutor.java   |  3 +
 .../impala/service/DescribeResultFactory.java      | 15 ++++-
 fe/src/main/jflex/sql-scanner.flex                 |  1 +
 .../org/apache/impala/analysis/AnalyzeDDLTest.java |  9 +++
 .../authorization/AuthorizationStmtTest.java       | 30 +++++-----
 .../queries/QueryTest/create-database.test         | 67 ++++++++++++++++++++++
 .../queries/QueryTest/describe-db.test             | 33 ++++++++++-
 .../queries/QueryTest/describe-hive-db.test        | 19 ++++++
 tests/metadata/test_metadata_query_statements.py   |  6 ++
 12 files changed, 197 insertions(+), 23 deletions(-)

diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
index 491055b..0b591b3 100644
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -135,6 +135,10 @@ struct TCreateDbParams {
   // The server name for security privileges when authorization is enabled.
   // TODO: Need to cleanup:IMPALA-7553
   6: optional string server_name
+
+  // Optional HDFS path for the database. Overrides location as the default location for
+  // all managed tables created in the database.
+  7: optional string managed_location
 }
 
 // Parameters of CREATE DATA SOURCE commands
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index 08903f9..4bf208e 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -296,7 +296,8 @@ terminal
   KW_IGNORE, KW_HAVING, KW_ICEBERG, KW_IF, KW_ILIKE, KW_IN, KW_INCREMENTAL, KW_INIT_FN, KW_INNER,
   KW_INPATH, KW_INSERT, KW_INT, KW_INTERMEDIATE, KW_INTERSECT, KW_INTERVAL, KW_INTO, KW_INVALIDATE, KW_IREGEXP,
   KW_IS, KW_JOIN,  KW_KUDU, KW_LAST, KW_LEFT, KW_LEXICAL, KW_LIKE, KW_LIMIT, KW_LINES,
-  KW_LOAD, KW_LOCATION, KW_LOGICAL_OR, KW_MAP, KW_MERGE_FN, KW_METADATA, KW_MINUS, KW_NORELY, KW_NOT,
+  KW_LOAD, KW_LOCATION, KW_LOGICAL_OR,
+  KW_MANAGED_LOCATION, KW_MAP, KW_MERGE_FN, KW_METADATA, KW_MINUS, KW_NORELY, KW_NOT,
   KW_NOVALIDATE, KW_NULL, KW_NULLS, KW_OFFSET, KW_ON, KW_OR, KW_ORC, KW_ORDER, KW_OUTER,
   KW_OVER, KW_OVERWRITE, KW_PARQUET, KW_PARQUETFILE, KW_PARTITION, KW_PARTITIONED,
   KW_PARTITIONS, KW_PRECEDING, KW_PREPARE_FN, KW_PRIMARY, KW_PRODUCED, KW_PURGE,
@@ -508,6 +509,7 @@ nonterminal Boolean if_exists_val;
 nonterminal Boolean if_not_exists_val;
 nonterminal Boolean is_primary_key_val;
 nonterminal HdfsUri location_val;
+nonterminal HdfsUri managed_location_val;
 nonterminal RowFormat row_format_val, opt_row_format_val;
 nonterminal String field_terminator_val;
 nonterminal String line_terminator_val;
@@ -1301,8 +1303,10 @@ opt_kw_column ::=
 
 create_db_stmt ::=
   KW_CREATE db_or_schema_kw if_not_exists_val:if_not_exists ident_or_default:db_name
-  opt_comment_val:comment location_val:location
-  {: RESULT = new CreateDbStmt(db_name, comment, location, if_not_exists); :}
+  opt_comment_val:comment location_val:location managed_location_val:managed_location
+  {: RESULT = new CreateDbStmt(db_name, comment, location,
+                               managed_location, if_not_exists);
+  :}
   ;
 
 
@@ -1866,6 +1870,13 @@ location_val ::=
   {: RESULT = null; :}
   ;
 
+managed_location_val ::=
+  KW_MANAGED_LOCATION STRING_LITERAL:location
+  {: RESULT = new HdfsUri(location); :}
+  | /* empty */
+  {: RESULT = null; :}
+  ;
+
 opt_init_string_val ::=
   LPAREN STRING_LITERAL:init_string RPAREN
   {: RESULT = init_string; :}
@@ -4013,6 +4024,8 @@ word ::=
   {: RESULT = r.toString(); :}
   | KW_LOGICAL_OR:r
   {: RESULT = r.toString(); :}
+  | KW_MANAGED_LOCATION:r
+  {: RESULT = r.toString(); :}
   | KW_MAP:r
   {: RESULT = r.toString(); :}
   | KW_MERGE_FN:r
diff --git a/fe/src/main/java/org/apache/impala/analysis/CreateDbStmt.java b/fe/src/main/java/org/apache/impala/analysis/CreateDbStmt.java
index aa43ade..538c314 100644
--- a/fe/src/main/java/org/apache/impala/analysis/CreateDbStmt.java
+++ b/fe/src/main/java/org/apache/impala/analysis/CreateDbStmt.java
@@ -32,6 +32,7 @@ import org.apache.impala.util.CatalogBlacklistUtils;
 public class CreateDbStmt extends StatementBase {
   private final String dbName_;
   private final HdfsUri location_;
+  private final HdfsUri managedLocation_;
   private final String comment_;
   private final boolean ifNotExists_;
   // Database owner. Set during analysis.
@@ -44,7 +45,7 @@ public class CreateDbStmt extends StatementBase {
    * Creates a database with the given name.
    */
   public CreateDbStmt(String dbName) {
-    this(dbName, null, null, false);
+    this(dbName, null, null, null, false);
   }
 
   /**
@@ -54,10 +55,11 @@ public class CreateDbStmt extends StatementBase {
    * unless the ifNotExists is true.
    */
   public CreateDbStmt(String dbName, String comment, HdfsUri location,
-      boolean ifNotExists) {
+       HdfsUri managedlocation, boolean ifNotExists) {
     this.dbName_ = dbName;
     this.comment_ = comment;
     this.location_ = location;
+    this.managedLocation_ = managedlocation;
     this.ifNotExists_ = ifNotExists;
   }
 
@@ -72,6 +74,9 @@ public class CreateDbStmt extends StatementBase {
     sb.append(dbName_);
     if (comment_ != null) sb.append(" COMMENT '" + comment_ + "'");
     if (location_ != null) sb.append(" LOCATION '" + location_ + "'");
+    if (managedLocation_ != null) {
+      sb.append(" MANAGEDLOCATION '" + managedLocation_ + "'");
+    }
     return sb.toString();
   }
 
@@ -80,6 +85,8 @@ public class CreateDbStmt extends StatementBase {
     params.setDb(getDb());
     params.setComment(getComment());
     params.setLocation(location_ == null ? null : location_.toString());
+    params.setManaged_location(
+        managedLocation_ == null ? null : managedLocation_.toString());
     params.setIf_not_exists(getIfNotExists());
     params.setOwner(getOwner());
     params.setServer_name(serverName_);
@@ -106,6 +113,9 @@ public class CreateDbStmt extends StatementBase {
     if (location_ != null) {
       location_.analyze(analyzer, Privilege.ALL, FsAction.READ_WRITE);
     }
+    if (managedLocation_ != null) {
+      managedLocation_.analyze(analyzer, Privilege.ALL, FsAction.READ_WRITE);
+    }
     owner_ = analyzer.getUserShortName();
     // Set the servername here if authorization is enabled because analyzer_ is not
     // available in the toThrift() method.
diff --git a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
index 8ef85af..ab71bf8 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -1333,6 +1333,9 @@ public class CatalogOpExecutor {
     if (params.getLocation() != null) {
       db.setLocationUri(params.getLocation());
     }
+    if (params.getManaged_location() != null) {
+      db.setManagedLocationUri(params.getManaged_location());
+    }
     db.setOwnerName(params.getOwner());
     db.setOwnerType(PrincipalType.USER);
     if (LOG.isTraceEnabled()) LOG.trace("Creating database " + dbName);
diff --git a/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java b/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
index 9a945f3..af0743d 100644
--- a/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
+++ b/fe/src/main/java/org/apache/impala/service/DescribeResultFactory.java
@@ -73,9 +73,11 @@ public class DescribeResultFactory {
     org.apache.hadoop.hive.metastore.api.Database msDb = db.getMetaStoreDb();
     descResult.results = Lists.newArrayList();
     String location = null;
+    String managedLocation = null;
     String comment = null;
     if(msDb != null) {
       location = msDb.getLocationUri();
+      managedLocation = msDb.getManagedLocationUri();
       comment = msDb.getDescription();
     }
 
@@ -85,8 +87,17 @@ public class DescribeResultFactory {
     dbLocationCol.setString_val(Objects.toString(location, ""));
     TColumnValue commentCol = new TColumnValue();
     commentCol.setString_val(Objects.toString(comment, ""));
-    descResult.results.add(
-        new TResultRow(Lists.newArrayList(dbNameCol, dbLocationCol, commentCol)));
+    descResult.results.add(new TResultRow(
+        Lists.newArrayList(dbNameCol, dbLocationCol, commentCol)));
+
+    if (managedLocation != null) {
+      TColumnValue keyCol = new TColumnValue();
+      keyCol.setString_val("managedlocation:");
+      TColumnValue dbManagedLocationCol = new TColumnValue();
+      dbManagedLocationCol.setString_val(Objects.toString(managedLocation, ""));
+      descResult.results.add(new TResultRow(
+          Lists.newArrayList(keyCol, dbManagedLocationCol, EMPTY)));
+    }
     return descResult;
   }
 
diff --git a/fe/src/main/jflex/sql-scanner.flex b/fe/src/main/jflex/sql-scanner.flex
index 16bb2a7..95d49e0 100644
--- a/fe/src/main/jflex/sql-scanner.flex
+++ b/fe/src/main/jflex/sql-scanner.flex
@@ -179,6 +179,7 @@ import org.apache.impala.thrift.TReservedWordsVersion;
     keywordMap.put("lines", SqlParserSymbols.KW_LINES);
     keywordMap.put("load", SqlParserSymbols.KW_LOAD);
     keywordMap.put("location", SqlParserSymbols.KW_LOCATION);
+    keywordMap.put("managedlocation", SqlParserSymbols.KW_MANAGED_LOCATION);
     keywordMap.put("map", SqlParserSymbols.KW_MAP);
     keywordMap.put("merge_fn", SqlParserSymbols.KW_MERGE_FN);
     keywordMap.put("metadata", SqlParserSymbols.KW_METADATA);
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
index 47ffe59..c0b33cc 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -1937,6 +1937,15 @@ public class AnalyzeDDLTest extends FrontendTestBase {
     AnalysisError("create database new_db location " +
         "'blah://bucket/test-warehouse/new_db'",
         "No FileSystem for scheme: blah");
+
+    // URIs for managedlocation.
+    AnalyzesOk("create database new_db managedlocation '/test-warehouse/new_db'");
+    AnalyzesOk("create database new_db location " +
+        "'/test-warehouse/new_db' managedlocation " +
+        "'/test-warehouse/new_db'");
+    AnalysisError("create database new_db managedlocation " +
+        "'blah://bucket/test-warehouse/new_db'",
+        "No FileSystem for scheme: blah");
   }
 
   @Test
diff --git a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
index 724ce09..23c843b 100644
--- a/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
+++ b/fe/src/test/java/org/apache/impala/authorization/AuthorizationStmtTest.java
@@ -1537,20 +1537,22 @@ public class AuthorizationStmtTest extends AuthorizationTestBase {
 
     // Create a database with a specific location.
     String uri = "hdfs://localhost:20500/test-warehouse/new_location";
-    String stmt = "create database newdb location " + "'" + uri + "'";
-    authorize(stmt)
-        .ok(onServer(TPrivilegeLevel.ALL))
-        .ok(onServer(TPrivilegeLevel.OWNER))
-        .ok(onServer(TPrivilegeLevel.CREATE), onUri(uri, TPrivilegeLevel.ALL))
-        .ok(onServer(TPrivilegeLevel.CREATE), onUri(uri, TPrivilegeLevel.OWNER))
-        .error(createError("newdb"))
-        .error(createError("newdb"), onServer(allExcept(TPrivilegeLevel.ALL,
-            TPrivilegeLevel.OWNER, TPrivilegeLevel.CREATE)),
-            onUri(uri, TPrivilegeLevel.ALL))
-        .error(createError("newdb"), onServer(allExcept(TPrivilegeLevel.ALL,
-            TPrivilegeLevel.OWNER, TPrivilegeLevel.CREATE)),
-            onUri(uri, TPrivilegeLevel.OWNER))
-        .error(accessError(uri), onServer(TPrivilegeLevel.CREATE));
+    for (AuthzTest test: new AuthzTest[]{
+      authorize("create database newdb location " + "'" + uri + "'"),
+      authorize("create database newdb managedlocation " + "'" + uri + "'")}) {
+      test.ok(onServer(TPrivilegeLevel.ALL))
+          .ok(onServer(TPrivilegeLevel.OWNER))
+          .ok(onServer(TPrivilegeLevel.CREATE), onUri(uri, TPrivilegeLevel.ALL))
+          .ok(onServer(TPrivilegeLevel.CREATE), onUri(uri, TPrivilegeLevel.OWNER))
+          .error(createError("newdb"))
+          .error(createError("newdb"), onServer(allExcept(TPrivilegeLevel.ALL,
+              TPrivilegeLevel.OWNER, TPrivilegeLevel.CREATE)),
+              onUri(uri, TPrivilegeLevel.ALL))
+          .error(createError("newdb"), onServer(allExcept(TPrivilegeLevel.ALL,
+              TPrivilegeLevel.OWNER, TPrivilegeLevel.CREATE)),
+              onUri(uri, TPrivilegeLevel.OWNER))
+          .error(accessError(uri), onServer(TPrivilegeLevel.CREATE));
+    }
 
     // Database already exists.
     for (AuthzTest test: new AuthzTest[]{
diff --git a/testdata/workloads/functional-query/queries/QueryTest/create-database.test b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
index 5cdaed3..e617a16 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/create-database.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/create-database.test
@@ -162,3 +162,70 @@ drop database $DATABASE_restrict restrict
 show databases like '$DATABASE_restrict'
 ---- RESULTS
 ====
+---- QUERY
+# Test CREATE DATABASE ... MANAGEDLOCATION
+create database if not exists $DATABASE_loc comment "For testing"
+  managedlocation '$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation'
+---- RESULTS
+'Database has been created.'
+====
+---- QUERY
+# Test that DESCRIBE shows the proper database location and managedlocation
+# for a newly created database with location
+describe database $DATABASE_loc
+---- RESULTS
+'$DATABASE_loc','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/$DATABASE_loc.db','For testing'
+'managedlocation:','$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+# Test that DESCRIBE EXTENDED also has all of the necessary info.
+describe database extended $DATABASE_loc
+---- RESULTS
+'$DATABASE_loc','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/$DATABASE_loc.db','For testing'
+'managedlocation:','$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation',''
+'Owner: ','',''
+'','$USER','USER'
+---- TYPES
+string, string, string
+====
+---- QUERY
+drop database $DATABASE_loc
+---- RESULTS
+'Database has been dropped.'
+====
+---- QUERY
+# Test CREATE DATABASE ... LOCATION ... MANAGEDLOCATION
+create database if not exists $DATABASE_loc comment "For testing"
+  location '$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/specified_location'
+  managedlocation '$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation'
+---- RESULTS
+'Database has been created.'
+====
+---- QUERY
+# Test that DESCRIBE shows the proper database location and  managedlocation
+# for a newly created database with location
+describe database $DATABASE_loc
+---- RESULTS
+'$DATABASE_loc','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/specified_location','For testing'
+'managedlocation:','$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+# Test that DESCRIBE EXTENDED also has all of the necessary info.
+describe database extended $DATABASE_loc
+---- RESULTS
+'$DATABASE_loc','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/specified_location','For testing'
+'managedlocation:','$NAMENODE/$MANAGED_WAREHOUSE_DIR/specified_managedlocation',''
+'Owner: ','',''
+'','$USER','USER'
+---- TYPES
+string, string, string
+====
+---- QUERY
+drop database $DATABASE_loc
+---- RESULTS
+'Database has been dropped.'
+====
\ No newline at end of file
diff --git a/testdata/workloads/functional-query/queries/QueryTest/describe-db.test b/testdata/workloads/functional-query/queries/QueryTest/describe-db.test
index d651e79..9df2cfa 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/describe-db.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/describe-db.test
@@ -44,11 +44,40 @@ describe database extended impala_test_desc_db4
 string, string, string
 ====
 ---- QUERY
-describe database formatted impala_test_desc_db4
+describe database impala_test_desc_db5
+---- RESULTS
+'impala_test_desc_db5','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/impala_test_desc_db5.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+describe database extended impala_test_desc_db5
 ---- RESULTS
 '','$USER','USER'
 'Owner: ','',''
-'impala_test_desc_db4','$NAMENODE/test2.db','test comment'
+'impala_test_desc_db5','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/impala_test_desc_db5.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+describe database extended impala_test_desc_db5
+---- RESULTS
+'','$USER','USER'
+'Owner: ','',''
+'impala_test_desc_db5','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/impala_test_desc_db5.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+describe database formatted impala_test_desc_db5
+---- RESULTS
+'','$USER','USER'
+'Owner: ','',''
+'impala_test_desc_db5','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/impala_test_desc_db5.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
 ---- TYPES
 string, string, string
 ====
diff --git a/testdata/workloads/functional-query/queries/QueryTest/describe-hive-db.test b/testdata/workloads/functional-query/queries/QueryTest/describe-hive-db.test
index ff7c8a3..8274210 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/describe-hive-db.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/describe-hive-db.test
@@ -28,3 +28,22 @@ describe database extended hive_test_desc_db
 ---- TYPES
 string, string, string
 ====
+---- QUERY
+# Test printing of database with managedlocation.
+describe database hive_test_desc_db2
+---- RESULTS
+'hive_test_desc_db2','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/hive_test_desc_db2.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
+---- TYPES
+string, string, string
+====
+---- QUERY
+describe database extended hive_test_desc_db2
+---- RESULTS
+'hive_test_desc_db2','$NAMENODE/$EXTERNAL_WAREHOUSE_DIR/hive_test_desc_db2.db','test comment'
+'managedlocation:','$NAMENODE/test2.db',''
+'','$USER','USER'
+'Owner: ','',''
+---- TYPES
+string, string, string
+====
diff --git a/tests/metadata/test_metadata_query_statements.py b/tests/metadata/test_metadata_query_statements.py
index eabb4b1..2979ce2 100644
--- a/tests/metadata/test_metadata_query_statements.py
+++ b/tests/metadata/test_metadata_query_statements.py
@@ -182,8 +182,12 @@ class TestMetadataQueryStatements(ImpalaTestSuite):
                           "location '" + get_fs_path("/testdb") + "'")
       self.client.execute("create database impala_test_desc_db4 comment 'test comment' "
                           "location \"" + get_fs_path("/test2.db") + "\"")
+      self.client.execute("create database impala_test_desc_db5 comment 'test comment' "
+                          "managedlocation \"" + get_fs_path("/test2.db") + "\"")
       self.run_stmt_in_hive("create database hive_test_desc_db comment 'test comment' "
                            "with dbproperties('pi' = '3.14', 'e' = '2.82')")
+      self.run_stmt_in_hive("create database hive_test_desc_db2 comment 'test comment' "
+                           "managedlocation '" + get_fs_path("/test2.db") + "'")
       if cluster_properties.is_event_polling_enabled():
         # Using HMS event processor - wait until the database shows up.
         self.wait_for_db_to_appear("hive_test_desc_db", timeout_s=30)
@@ -198,7 +202,9 @@ class TestMetadataQueryStatements(ImpalaTestSuite):
 
   def __test_describe_db_cleanup(self):
     self.cleanup_db('hive_test_desc_db')
+    self.cleanup_db('hive_test_desc_db2')
     self.cleanup_db('impala_test_desc_db1')
     self.cleanup_db('impala_test_desc_db2')
     self.cleanup_db('impala_test_desc_db3')
     self.cleanup_db('impala_test_desc_db4')
+    self.cleanup_db('impala_test_desc_db5')