You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2018/05/16 02:53:14 UTC

[07/12] impala git commit: IMPALA-6916: Implement COMMENT ON DATABASE

IMPALA-6916: Implement COMMENT ON DATABASE

This patch implements updating comment on a database.

Syntax:
COMMENT ON DATABASE db IS 'comment'

Testing:
- Added new front-end tests
- Ran all front-end tests
- Added new end-to-end tests
- Ran end-to-end DDL tests

Change-Id: Ifcf909c18f97073346f6f603538bf921e69fbb00
Reviewed-on: http://gerrit.cloudera.org:8080/10171
Reviewed-by: Alex Behm <al...@cloudera.com>
Tested-by: Impala Public Jenkins <im...@cloudera.com>


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

Branch: refs/heads/2.x
Commit: 0ce9056a1297f741a9c8160c322109a4a45047e8
Parents: 5e32c08
Author: Fredy Wijaya <fw...@cloudera.com>
Authored: Mon Apr 23 20:48:49 2018 -0700
Committer: Impala Public Jenkins <im...@gerrit.cloudera.org>
Committed: Tue May 15 21:10:11 2018 +0000

----------------------------------------------------------------------
 common/thrift/CatalogService.thrift             |  3 ++
 common/thrift/JniCatalog.thrift                 |  8 +++
 fe/src/main/cup/sql-parser.cup                  | 17 ++++++-
 .../apache/impala/analysis/AnalysisContext.java |  9 +++-
 .../apache/impala/analysis/CommentOnDbStmt.java | 47 ++++++++++++++++++
 .../apache/impala/analysis/CommentOnStmt.java   | 37 ++++++++++++++
 .../impala/service/CatalogOpExecutor.java       | 51 ++++++++++++++++++++
 .../org/apache/impala/service/Frontend.java     | 10 ++++
 .../apache/impala/analysis/AnalyzeDDLTest.java  |  9 ++++
 .../impala/analysis/AuthorizationTest.java      | 11 +++++
 .../org/apache/impala/analysis/ParserTest.java  | 15 ++++--
 tests/metadata/test_ddl.py                      | 16 ++++++
 tests/metadata/test_ddl_base.py                 |  5 ++
 13 files changed, 233 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/common/thrift/CatalogService.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/CatalogService.thrift b/common/thrift/CatalogService.thrift
index 8a93998..6c02722 100644
--- a/common/thrift/CatalogService.thrift
+++ b/common/thrift/CatalogService.thrift
@@ -129,6 +129,9 @@ struct TDdlExecRequest {
 
   // True if SYNC_DDL is set in query options
   22: required bool sync_ddl
+
+  // Parameters for COMMENT ON
+  23: optional JniCatalog.TCommentOnParams comment_on_params
 }
 
 // Response from executing a TDdlExecRequest

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/common/thrift/JniCatalog.thrift
----------------------------------------------------------------------
diff --git a/common/thrift/JniCatalog.thrift b/common/thrift/JniCatalog.thrift
index 371475b..d3bd7a0 100644
--- a/common/thrift/JniCatalog.thrift
+++ b/common/thrift/JniCatalog.thrift
@@ -52,6 +52,7 @@ enum TDdlType {
   GRANT_PRIVILEGE,
   REVOKE_PRIVILEGE,
   TRUNCATE_TABLE,
+  COMMENT_ON
 }
 
 // Types of ALTER TABLE commands supported.
@@ -630,3 +631,10 @@ struct TGetCatalogUsageResponse{
   2: required list<TTableUsageMetrics> frequently_accessed_tables
 }
 
+struct TCommentOnParams {
+  // Name of comment to alter. When this field is not set, the comment will be removed.
+  1: optional string comment
+
+  // Name of database to alter.
+  2: optional string db
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/cup/sql-parser.cup
----------------------------------------------------------------------
diff --git a/fe/src/main/cup/sql-parser.cup b/fe/src/main/cup/sql-parser.cup
index f2d7cef..261f8ec 100644
--- a/fe/src/main/cup/sql-parser.cup
+++ b/fe/src/main/cup/sql-parser.cup
@@ -409,6 +409,7 @@ nonterminal Qualifier union_op;
 
 nonterminal PartitionDef partition_def;
 nonterminal List<PartitionDef> partition_def_list;
+nonterminal CommentOnStmt comment_on_stmt;
 nonterminal AlterTableStmt alter_tbl_stmt;
 nonterminal StatementBase alter_view_stmt;
 nonterminal ComputeStatsStmt compute_stats_stmt;
@@ -443,7 +444,7 @@ nonterminal ArrayList<StructField> struct_field_def_list;
 // Options for DDL commands - CREATE/DROP/ALTER
 nonterminal HdfsCachingOp cache_op_val, opt_cache_op_val;
 nonterminal BigDecimal opt_cache_op_replication;
-nonterminal String comment_val, opt_comment_val;
+nonterminal String comment_val, opt_comment_val, nullable_comment_val;
 nonterminal Boolean external_val;
 nonterminal Boolean purge_val;
 nonterminal String opt_init_string_val;
@@ -650,6 +651,8 @@ stmt ::=
   {: RESULT = grant_privilege; :}
   | revoke_privilege_stmt:revoke_privilege
   {: RESULT = revoke_privilege; :}
+  | comment_on_stmt:comment_on
+  {: RESULT = comment_on; :}
   | stmt:s SEMICOLON
   {: RESULT = s; :}
   ;
@@ -1009,6 +1012,11 @@ partition_def_list ::=
   :}
   ;
 
+comment_on_stmt ::=
+  KW_COMMENT KW_ON KW_DATABASE ident_or_default:db_name KW_IS nullable_comment_val:comment
+  {: RESULT = new CommentOnDbStmt(db_name, comment); :}
+  ;
+
 alter_tbl_stmt ::=
   KW_ALTER KW_TABLE table_name:table replace_existing_cols_val:replace KW_COLUMNS
   LPAREN column_def_list:col_defs RPAREN
@@ -1474,6 +1482,13 @@ opt_comment_val ::=
   {: RESULT = null; :}
   ;
 
+nullable_comment_val ::=
+  STRING_LITERAL:comment
+  {: RESULT = comment; :}
+  | KW_NULL
+  {: RESULT = null; :}
+  ;
+
 location_val ::=
   KW_LOCATION STRING_LITERAL:location
   {: RESULT = new HdfsUri(location); :}

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
index c886079..791e528 100644
--- a/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
+++ b/fe/src/main/java/org/apache/impala/analysis/AnalysisContext.java
@@ -149,6 +149,7 @@ public class AnalysisContext {
     public UpdateStmt getUpdateStmt() { return (UpdateStmt) stmt_; }
     public boolean isDeleteStmt() { return stmt_ instanceof DeleteStmt; }
     public DeleteStmt getDeleteStmt() { return (DeleteStmt) stmt_; }
+    public boolean isCommentOnStmt() { return stmt_ instanceof CommentOnStmt; }
 
     public boolean isCatalogOp() {
       return isUseStmt() || isViewMetadataStmt() || isDdlStmt();
@@ -161,7 +162,8 @@ public class AnalysisContext {
           isAlterViewStmt() || isComputeStatsStmt() || isCreateUdfStmt() ||
           isCreateUdaStmt() || isDropFunctionStmt() || isCreateTableAsSelectStmt() ||
           isCreateDataSrcStmt() || isDropDataSrcStmt() || isDropStatsStmt() ||
-          isCreateDropRoleStmt() || isGrantRevokeStmt() || isTruncateStmt();
+          isCreateDropRoleStmt() || isGrantRevokeStmt() || isTruncateStmt() ||
+          isCommentOnStmt();
     }
 
     private boolean isViewMetadataStmt() {
@@ -349,6 +351,11 @@ public class AnalysisContext {
       return (ShowCreateFunctionStmt) stmt_;
     }
 
+    public CommentOnStmt getCommentOnStmt() {
+      Preconditions.checkState(isCommentOnStmt());
+      return (CommentOnStmt) stmt_;
+    }
+
     public StatementBase getStmt() { return stmt_; }
     public Analyzer getAnalyzer() { return analyzer_; }
     public Set<TAccessEvent> getAccessEvents() { return analyzer_.getAccessEvents(); }

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/java/org/apache/impala/analysis/CommentOnDbStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CommentOnDbStmt.java b/fe/src/main/java/org/apache/impala/analysis/CommentOnDbStmt.java
new file mode 100644
index 0000000..8f622fd
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/CommentOnDbStmt.java
@@ -0,0 +1,47 @@
+// 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.impala.analysis;
+
+import com.google.common.base.Preconditions;
+import org.apache.impala.authorization.Privilege;
+import org.apache.impala.common.AnalysisException;
+import org.apache.impala.thrift.TCommentOnParams;
+
+/**
+ * Represents a COMMENT ON DATABASE db IS 'comment' statement.
+ */
+public class CommentOnDbStmt extends CommentOnStmt {
+  private final String dbName_;
+
+  public CommentOnDbStmt(String dbName, String comment) {
+    super(comment);
+    Preconditions.checkNotNull(dbName);
+    dbName_ = dbName;
+  }
+
+  @Override
+  public void analyze(Analyzer analyzer) throws AnalysisException {
+    analyzer.getDb(dbName_, Privilege.ALTER);
+  }
+
+  public TCommentOnParams toThrift() {
+    TCommentOnParams params = super.toThrift();
+    params.setDb(dbName_);
+    return params;
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/java/org/apache/impala/analysis/CommentOnStmt.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/analysis/CommentOnStmt.java b/fe/src/main/java/org/apache/impala/analysis/CommentOnStmt.java
new file mode 100644
index 0000000..dffc08d
--- /dev/null
+++ b/fe/src/main/java/org/apache/impala/analysis/CommentOnStmt.java
@@ -0,0 +1,37 @@
+// 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.impala.analysis;
+
+import org.apache.impala.thrift.TCommentOnParams;
+
+/**
+ * A base class for COMMENT ON statement.
+ */
+public class CommentOnStmt extends StatementBase {
+  protected final String comment_;
+
+  public CommentOnStmt(String comment) {
+    comment_ = comment;
+  }
+
+  public TCommentOnParams toThrift() {
+    TCommentOnParams params = new TCommentOnParams();
+    params.setComment(comment_);
+    return params;
+  }
+}

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
----------------------------------------------------------------------
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 fdee124..ea67389 100644
--- a/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
+++ b/fe/src/main/java/org/apache/impala/service/CatalogOpExecutor.java
@@ -39,6 +39,7 @@ import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsData;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsDesc;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.DecimalColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.DoubleColumnStatsData;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
@@ -109,6 +110,7 @@ import org.apache.impala.thrift.TColumn;
 import org.apache.impala.thrift.TColumnStats;
 import org.apache.impala.thrift.TColumnType;
 import org.apache.impala.thrift.TColumnValue;
+import org.apache.impala.thrift.TCommentOnParams;
 import org.apache.impala.thrift.TCreateDataSourceParams;
 import org.apache.impala.thrift.TCreateDbParams;
 import org.apache.impala.thrift.TCreateDropRoleParams;
@@ -325,6 +327,9 @@ public class CatalogOpExecutor {
         grantRevokeRolePrivilege(requestingUser,
             ddlRequest.getGrant_revoke_priv_params(), response);
         break;
+      case COMMENT_ON:
+        alterCommentOn(ddlRequest.getComment_on_params(), response);
+        break;
       default: throw new IllegalStateException("Unexpected DDL exec request type: " +
           ddlRequest.ddl_type);
     }
@@ -3503,4 +3508,50 @@ public class CatalogOpExecutor {
     Preconditions.checkState(tbl.isLoaded());
     return tbl;
   }
+
+  private void alterCommentOn(TCommentOnParams params, TDdlExecResponse response)
+      throws ImpalaRuntimeException, CatalogException, InternalException {
+    if (params.getDb() != null) {
+      alterCommentOnDb(params.getDb(), params.getComment(), response);
+    }
+  }
+
+  private void alterCommentOnDb(String dbName, String comment, TDdlExecResponse response)
+      throws ImpalaRuntimeException, CatalogException, InternalException {
+    Db db = catalog_.getDb(dbName);
+    if (db == null) {
+      throw new CatalogException("Database: " + db.getName() + " does not exist.");
+    }
+    synchronized (metastoreDdlLock_) {
+      Database msDb = db.getMetaStoreDb();
+      String originalComment = msDb.getDescription();
+      msDb.setDescription(comment);
+      try {
+        applyAlterDatabase(db);
+      } catch (ImpalaRuntimeException e) {
+        msDb.setDescription(originalComment);
+        throw e;
+      }
+    }
+    addDbToCatalogUpdate(db, response.result);
+    addSummary(response, "Updated database.");
+  }
+
+  private void addDbToCatalogUpdate(Db db, TCatalogUpdateResult result) {
+    Preconditions.checkNotNull(db);
+    // Updating the new catalog version and setting it to the DB catalog version while
+    // holding the catalog version lock for an atomic operation. Most DB operations are
+    // short-lived. It is unnecessary to have a fine-grained DB lock.
+    catalog_.getLock().writeLock().lock();
+    try {
+      long newCatalogVersion = catalog_.incrementAndGetCatalogVersion();
+      db.setCatalogVersion(newCatalogVersion);
+      TCatalogObject updatedCatalogObject = db.toTCatalogObject();
+      updatedCatalogObject.setCatalog_version(newCatalogVersion);
+      result.addToUpdated_catalog_objects(updatedCatalogObject);
+      result.setVersion(updatedCatalogObject.getCatalog_version());
+    } finally {
+      catalog_.getLock().writeLock().unlock();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/main/java/org/apache/impala/service/Frontend.java
----------------------------------------------------------------------
diff --git a/fe/src/main/java/org/apache/impala/service/Frontend.java b/fe/src/main/java/org/apache/impala/service/Frontend.java
index f9b7502..5e51c08 100644
--- a/fe/src/main/java/org/apache/impala/service/Frontend.java
+++ b/fe/src/main/java/org/apache/impala/service/Frontend.java
@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.impala.analysis.AnalysisContext;
 import org.apache.impala.analysis.AnalysisContext.AnalysisResult;
+import org.apache.impala.analysis.CommentOnStmt;
 import org.apache.impala.analysis.CreateDataSrcStmt;
 import org.apache.impala.analysis.CreateDropRoleStmt;
 import org.apache.impala.analysis.CreateUdaStmt;
@@ -98,6 +99,7 @@ import org.apache.impala.thrift.TCatalogOpType;
 import org.apache.impala.thrift.TCatalogServiceRequestHeader;
 import org.apache.impala.thrift.TColumn;
 import org.apache.impala.thrift.TColumnValue;
+import org.apache.impala.thrift.TCommentOnParams;
 import org.apache.impala.thrift.TCreateDropRoleParams;
 import org.apache.impala.thrift.TDdlExecRequest;
 import org.apache.impala.thrift.TDdlType;
@@ -497,6 +499,14 @@ public class Frontend {
       req.setGrant_revoke_priv_params(params);
       ddl.op_type = TCatalogOpType.DDL;
       ddl.setDdl_params(req);
+    } else if (analysis.isCommentOnStmt()) {
+      CommentOnStmt commentOnStmt = analysis.getCommentOnStmt();
+      TCommentOnParams params = commentOnStmt.toThrift();
+      TDdlExecRequest req = new TDdlExecRequest();
+      req.setDdl_type(TDdlType.COMMENT_ON);
+      req.setComment_on_params(params);
+      ddl.op_type = TCatalogOpType.DDL;
+      ddl.setDdl_params(req);
     } else {
       throw new IllegalStateException("Unexpected CatalogOp statement type.");
     }

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
----------------------------------------------------------------------
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 8f9e384..3a33c4e 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeDDLTest.java
@@ -3882,4 +3882,13 @@ public class AnalyzeDDLTest extends FrontendTestBase {
       }
     }
   }
+
+  @Test
+  public void TestCommentOn() {
+    AnalyzesOk("comment on database functional is 'comment'");
+    AnalyzesOk("comment on database functional is ''");
+    AnalyzesOk("comment on database functional is null");
+    AnalysisError("comment on database doesntexist is 'comment'",
+        "Database does not exist: doesntexist");
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
index e5b7702..d4bddf5 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AuthorizationTest.java
@@ -2801,6 +2801,17 @@ public class AuthorizationTest extends FrontendTestBase {
     }
   }
 
+  @Test
+  public void TestCommentOn() throws ImpalaException {
+    // User has ALTER privilege on functional_text_lzo database.
+    AuthzOk("comment on database functional_text_lzo is 'comment'");
+    // User does not have ALTER privilege on functional database.
+    AuthzError("comment on database functional is 'comment'",
+        "User '%s' does not have privileges to execute 'ALTER' on: functional");
+    AuthzError("comment on database doesntexist is 'comment'",
+        "User '%s' does not have privileges to execute 'ALTER' on: doesntexist");
+  }
+
   private void TestWithIncorrectConfig(AuthorizationConfig authzConfig, User user)
       throws ImpalaException {
     Frontend fe = new Frontend(authzConfig, ctx_.catalog);

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
----------------------------------------------------------------------
diff --git a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
index ab0eb78..b5082db 100644
--- a/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/ParserTest.java
@@ -3233,9 +3233,9 @@ public class ParserTest extends FrontendTestBase {
         "c, b, c from t\n" +
         "^\n" +
         "Encountered: IDENTIFIER\n" +
-        "Expected: ALTER, COMPUTE, CREATE, DELETE, DESCRIBE, DROP, EXPLAIN, GRANT, " +
-        "INSERT, INVALIDATE, LOAD, REFRESH, REVOKE, SELECT, SET, SHOW, TRUNCATE, " +
-        "UPDATE, UPSERT, USE, VALUES, WITH\n");
+        "Expected: ALTER, COMMENT, COMPUTE, CREATE, DELETE, DESCRIBE, DROP, EXPLAIN, " +
+        "GRANT, INSERT, INVALIDATE, LOAD, REFRESH, REVOKE, SELECT, SET, SHOW, " +
+        "TRUNCATE, UPDATE, UPSERT, USE, VALUES, WITH\n");
 
     // missing select list
     ParserError("select from t",
@@ -3736,4 +3736,13 @@ public class ParserTest extends FrontendTestBase {
     ParsesOk("DESCRIBE functional.alltypes;");
     ParsesOk("SET num_nodes=1;");
   }
+
+  @Test
+  public void TestCommentOn() {
+    ParsesOk("COMMENT ON DATABASE db IS 'comment'");
+    ParsesOk("COMMENT ON DATABASE db IS ''");
+    ParsesOk("COMMENT ON DATABASE db IS NULL");
+    ParserError("COMMENT ON DATABASE IS 'comment'");
+    ParserError("COMMENT ON DATABASE db IS");
+  }
 }

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/tests/metadata/test_ddl.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl.py b/tests/metadata/test_ddl.py
index 0830060..c2f34a9 100644
--- a/tests/metadata/test_ddl.py
+++ b/tests/metadata/test_ddl.py
@@ -194,6 +194,22 @@ class TestDdlStatements(TestDdlBase):
     self.run_test_case('QueryTest/create-database', vector, use_db=unique_database,
         multiple_impalad=self._use_multiple_impalad(vector))
 
+  def test_comment_on_database(self, vector, unique_database):
+    comment = self._get_db_comment(unique_database)
+    assert '' == comment
+
+    self.client.execute("comment on database {0} is 'comment'".format(unique_database))
+    comment = self._get_db_comment(unique_database)
+    assert 'comment' == comment
+
+    self.client.execute("comment on database {0} is ''".format(unique_database))
+    comment = self._get_db_comment(unique_database)
+    assert '' == comment
+
+    self.client.execute("comment on database {0} is null".format(unique_database))
+    comment = self._get_db_comment(unique_database)
+    assert '' == comment
+
   # There is a query in QueryTest/create-table that references nested types, which is not
   # supported if old joins and aggs are enabled. Since we do not get any meaningful
   # additional coverage by running a DDL test under the old aggs and joins, it can be

http://git-wip-us.apache.org/repos/asf/impala/blob/0ce9056a/tests/metadata/test_ddl_base.py
----------------------------------------------------------------------
diff --git a/tests/metadata/test_ddl_base.py b/tests/metadata/test_ddl_base.py
index 3044ef0..0b0968d 100644
--- a/tests/metadata/test_ddl_base.py
+++ b/tests/metadata/test_ddl_base.py
@@ -87,3 +87,8 @@ class TestDdlBase(ImpalaTestSuite):
           break
         properties[row[1].rstrip()] = row[2].rstrip()
     return properties
+
+  def _get_db_comment(self, db_name):
+    """Extracts the DB comment from the output of DESCRIBE DATABASE"""
+    result = self.client.execute("describe database {0}".format(db_name))
+    return result.data[0].split('\t')[2]
\ No newline at end of file