You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by se...@apache.org on 2015/09/18 22:35:39 UTC

[34/41] hive git commit: HIVE-11706 : Implement show create database (Navis via Ashutosh Chauhan)

HIVE-11706 : Implement show create database (Navis via Ashutosh Chauhan)

Signed-off-by: Ashutosh Chauhan <ha...@apache.org>


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

Branch: refs/heads/llap
Commit: 3cf7bd9e871e7f258d764b2d988cabfb356b6c71
Parents: 8da2ed3
Author: Navis Ryu <na...@apache.org>
Authored: Tue Sep 8 22:01:00 2015 -0800
Committer: Ashutosh Chauhan <ha...@apache.org>
Committed: Thu Sep 17 13:35:17 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hive/ql/exec/DDLTask.java | 269 ++++++++-----------
 .../hive/ql/parse/DDLSemanticAnalyzer.java      |  17 ++
 .../apache/hadoop/hive/ql/parse/HiveParser.g    |   7 +-
 .../hive/ql/parse/SemanticAnalyzerFactory.java  |   2 +
 .../org/apache/hadoop/hive/ql/plan/DDLWork.java |  21 ++
 .../hadoop/hive/ql/plan/HiveOperation.java      |   1 +
 .../hive/ql/plan/ShowCreateDatabaseDesc.java    |  94 +++++++
 .../authorization/plugin/HiveOperationType.java |   1 +
 .../plugin/sqlstd/Operation2Privilege.java      |   2 +
 .../clientpositive/show_create_database.q       |   3 +
 .../clientpositive/show_create_database.q.out   |  19 ++
 .../tez/show_create_database.q.out              |  19 ++
 12 files changed, 296 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
index 734742c..210736b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
@@ -163,6 +163,7 @@ import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCompactionsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowConfDesc;
+import org.apache.hadoop.hive.ql.plan.ShowCreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -440,6 +441,11 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         return showPartitions(db, showParts);
       }
 
+      ShowCreateDatabaseDesc showCreateDb = work.getShowCreateDbDesc();
+      if (showCreateDb != null) {
+        return showCreateDatabase(db, showCreateDb);
+      }
+
       ShowCreateTableDesc showCreateTbl = work.getShowCreateTblDesc();
       if (showCreateTbl != null) {
         return showCreateTable(db, showCreateTbl);
@@ -545,9 +551,23 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     return 0;
   }
 
-  private DataOutputStream getOutputStream(Path outputFile) throws Exception {
-    FileSystem fs = outputFile.getFileSystem(conf);
-    return fs.create(outputFile);
+  private DataOutputStream getOutputStream(String resFile) throws HiveException {
+    try {
+      return getOutputStream(new Path(resFile));
+    } catch (HiveException e) {
+      throw e;
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
+  }
+
+  private DataOutputStream getOutputStream(Path outputFile) throws HiveException {
+    try {
+      FileSystem fs = outputFile.getFileSystem(conf);
+      return fs.create(outputFile);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    }
   }
 
   /**
@@ -1891,16 +1911,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showParts.getResFile());
     try {
-      Path resFile = new Path(showParts.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       formatter.showTablePartitions(outStream, parts);
-
-      outStream.close();
-      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "show partitions for table " + tabName);
     } finally {
@@ -1918,6 +1931,40 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       "NULL DEFINED AS"
   };
 
+  private int showCreateDatabase(Hive db, ShowCreateDatabaseDesc showCreateDb) throws HiveException {
+    DataOutputStream outStream = getOutputStream(showCreateDb.getResFile());
+    try {
+      String dbName = showCreateDb.getDatabaseName();
+      return showCreateDatabase(db, outStream, dbName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    } finally {
+      IOUtils.closeStream(outStream);
+    }
+  }
+
+  private int showCreateDatabase(Hive db, DataOutputStream outStream, String databaseName)
+      throws Exception {
+    Database database = db.getDatabase(databaseName);
+
+    StringBuilder createDb_str = new StringBuilder();
+    createDb_str.append("CREATE DATABASE `").append(database.getName()).append("`\n");
+    if (database.getDescription() != null) {
+      createDb_str.append("COMMENT\n  '");
+      createDb_str.append(escapeHiveCommand(database.getDescription())).append("'\n");
+    }
+    createDb_str.append("LOCATION\n  '");
+    createDb_str.append(database.getLocationUri()).append("'\n");
+    String propertiesToString = propertiesToString(database.getParameters(), null);
+    if (!propertiesToString.isEmpty()) {
+      createDb_str.append("WITH DBPROPERTIES (\n");
+      createDb_str.append(propertiesToString).append(")\n");
+    }
+
+    outStream.write(createDb_str.toString().getBytes("UTF-8"));
+    return 0;
+  }
+
   /**
    * Write a statement of how to create a table to a file.
    *
@@ -1931,6 +1978,19 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
    */
   private int showCreateTable(Hive db, ShowCreateTableDesc showCreateTbl) throws HiveException {
     // get the create table statement for the table and populate the output
+    DataOutputStream outStream = getOutputStream(showCreateTbl.getResFile());
+    try {
+      String tableName = showCreateTbl.getTableName();
+      return showCreateTable(db, outStream, tableName);
+    } catch (Exception e) {
+      throw new HiveException(e);
+    } finally {
+      IOUtils.closeStream(outStream);
+    }
+  }
+
+  private int showCreateTable(Hive db, DataOutputStream outStream, String tableName)
+      throws HiveException {
     final String EXTERNAL = "external";
     final String TEMPORARY = "temporary";
     final String LIST_COLUMNS = "columns";
@@ -1943,22 +2003,14 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     boolean needsLocation = true;
     StringBuilder createTab_str = new StringBuilder();
 
-    String tableName = showCreateTbl.getTableName();
     Table tbl = db.getTable(tableName, false);
-    DataOutputStream outStream = null;
     List<String> duplicateProps = new ArrayList<String>();
     try {
-      Path resFile = new Path(showCreateTbl.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       needsLocation = doesTableNeedLocation(tbl);
 
       if (tbl.isView()) {
         String createTab_stmt = "CREATE VIEW `" + tableName + "` AS " + tbl.getViewExpandedText();
         outStream.writeBytes(createTab_stmt.toString());
-        outStream.close();
-        outStream = null;
         return 0;
       }
 
@@ -2115,18 +2167,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       String tbl_location = "  '" + escapeHiveCommand(sd.getLocation()) + "'";
 
       // Table properties
-      String tbl_properties = "";
-      if (!tbl.getParameters().isEmpty()) {
-        Map<String, String> properties = new TreeMap<String, String>(tbl.getParameters());
-        List<String> realProps = new ArrayList<String>();
-        for (String key : properties.keySet()) {
-          if (properties.get(key) != null && !duplicateProps.contains(key)) {
-            realProps.add("  '" + key + "'='" +
-                escapeHiveCommand(StringEscapeUtils.escapeJava(properties.get(key))) + "'");
-          }
-        }
-        tbl_properties += StringUtils.join(realProps, ", \n");
-      }
+      String tbl_properties = propertiesToString(tbl.getParameters(), duplicateProps);
 
       createTab_stmt.add(TEMPORARY, tbl_temp);
       createTab_stmt.add(EXTERNAL, tbl_external);
@@ -2142,23 +2183,30 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       createTab_stmt.add(TBL_PROPERTIES, tbl_properties);
 
       outStream.writeBytes(createTab_stmt.render());
-      outStream.close();
-      outStream = null;
-    } catch (FileNotFoundException e) {
-      LOG.info("show create table: " + stringifyException(e));
-      return 1;
     } catch (IOException e) {
       LOG.info("show create table: " + stringifyException(e));
       return 1;
-    } catch (Exception e) {
-      throw new HiveException(e);
-    } finally {
-      IOUtils.closeStream(outStream);
     }
 
     return 0;
   }
 
+  private String propertiesToString(Map<String, String> props, List<String> exclude) {
+    String prop_string = "";
+    if (!props.isEmpty()) {
+      Map<String, String> properties = new TreeMap<String, String>(props);
+      List<String> realProps = new ArrayList<String>();
+      for (String key : properties.keySet()) {
+        if (properties.get(key) != null && (exclude == null || !exclude.contains(key))) {
+          realProps.add("  '" + key + "'='" +
+              escapeHiveCommand(StringEscapeUtils.escapeJava(properties.get(key))) + "'");
+        }
+      }
+      prop_string += StringUtils.join(realProps, ", \n");
+    }
+    return prop_string;
+  }
+
   private boolean containsNonNull(String[] values) {
     for (String value : values) {
       if (value != null) {
@@ -2202,12 +2250,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     indexes = db.getIndexes(tbl.getDbName(), tbl.getTableName(), (short) -1);
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showIndexes.getResFile());
     try {
-      Path resFile = new Path(showIndexes.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       if (showIndexes.isFormatted()) {
         // column headers
         outStream.writeBytes(MetaDataFormatUtils.getIndexColumnsHeader());
@@ -2219,10 +2263,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       {
         outStream.writeBytes(MetaDataFormatUtils.getAllColumnsInformation(index));
       }
-
-      outStream.close();
-      outStream = null;
-
     } catch (FileNotFoundException e) {
       LOG.info("show indexes: " + stringifyException(e));
       throw new HiveException(e.toString());
@@ -2259,15 +2299,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     LOG.info("results : " + databases.size());
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showDatabasesDesc.getResFile());
     try {
-      Path resFile = new Path(showDatabasesDesc.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       formatter.showDatabases(outStream, databases);
-      outStream.close();
-      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "show databases");
     } finally {
@@ -2304,16 +2338,10 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showTbls.getResFile());
     try {
-      Path resFile = new Path(showTbls.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       SortedSet<String> sortedTbls = new TreeSet<String>(tbls);
       formatter.showTables(outStream, sortedTbls);
-      outStream.close();
-      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "in database" + dbName);
     } finally {
@@ -2328,12 +2356,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     Table table = db.getTable(showCols.getTableName());
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showCols.getResFile());;
     try {
-      Path resFile = new Path(showCols.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       List<FieldSchema> cols = table.getCols();
       cols.addAll(table.getPartCols());
       // In case the query is served by HiveServer2, don't pad it with spaces,
@@ -2341,8 +2365,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       boolean isOutputPadded = !SessionState.get().isHiveServerQuery();
       outStream.writeBytes(MetaDataFormatUtils.getAllColumnsInformation(
           cols, false, isOutputPadded, null));
-      outStream.close();
-      outStream = null;
     } catch (IOException e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
     } finally {
@@ -2377,11 +2399,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showFuncs.getResFile());
     try {
-      Path resFile = new Path(showFuncs.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
       SortedSet<String> sortedFuncs = new TreeSet<String>(funcs);
       // To remove the primitive types
       sortedFuncs.removeAll(serdeConstants.PrimitiveTypes);
@@ -2392,8 +2411,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         outStream.writeBytes(iterFuncs.next());
         outStream.write(terminator);
       }
-      outStream.close();
-      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show function: " + stringifyException(e));
       return 1;
@@ -2430,11 +2447,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showLocks.getResFile());
     try {
-      Path resFile = new Path(showLocks.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
       List<HiveLock> locks = null;
 
       if (showLocks.getTableName() == null) {
@@ -2490,8 +2504,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         }
         outStream.write(terminator);
       }
-      outStream.close();
-      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show function: " + stringifyException(e));
       return 1;
@@ -2518,12 +2530,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     ShowLocksResponse rsp = lockMgr.getLocks();
 
     // write the results in the file
-    DataOutputStream os = null;
+    DataOutputStream os = getOutputStream(showLocks.getResFile());
     try {
-      Path resFile = new Path(showLocks.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      os = fs.create(resFile);
-
       // Write a header
       os.writeBytes("Lock ID");
       os.write(separator);
@@ -2577,9 +2585,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         }
 
       }
-
-      os.close();
-      os = null;
     } catch (FileNotFoundException e) {
       LOG.warn("show function: " + stringifyException(e));
       return 1;
@@ -2599,12 +2604,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     ShowCompactResponse rsp = db.showCompactions();
 
     // Write the results into the file
-    DataOutputStream os = null;
+    DataOutputStream os = getOutputStream(desc.getResFile());
     try {
-      Path resFile = new Path(desc.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      os = fs.create(resFile);
-
       // Write a header
       os.writeBytes("Database");
       os.write(separator);
@@ -2641,7 +2642,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           os.write(terminator);
         }
       }
-      os.close();
     } catch (IOException e) {
       LOG.warn("show compactions: " + stringifyException(e));
       return 1;
@@ -2656,12 +2656,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     GetOpenTxnsInfoResponse rsp = db.showTransactions();
 
     // Write the results into the file
-    DataOutputStream os = null;
+    DataOutputStream os = getOutputStream(desc.getResFile());
     try {
-      Path resFile = new Path(desc.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      os = fs.create(resFile);
-
       // Write a header
       os.writeBytes("Transaction ID");
       os.write(separator);
@@ -2682,7 +2678,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
         os.writeBytes(txn.getHostname());
         os.write(terminator);
       }
-      os.close();
     } catch (IOException e) {
       LOG.warn("show transactions: " + stringifyException(e));
       return 1;
@@ -2763,12 +2758,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     String funcName = descFunc.getName();
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(descFunc.getResFile());
     try {
-      Path resFile = new Path(descFunc.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       // get the function documentation
       Description desc = null;
       Class<?> funcClass = null;
@@ -2801,9 +2792,6 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
       }
 
       outStream.write(terminator);
-
-      outStream.close();
-      outStream = null;
     } catch (FileNotFoundException e) {
       LOG.warn("describe function: " + stringifyException(e));
       return 1;
@@ -2819,12 +2807,8 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
   }
 
   private int descDatabase(DescDatabaseDesc descDatabase) throws HiveException {
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(descDatabase.getResFile());
     try {
-      Path resFile = new Path(descDatabase.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       Database database = db.getDatabase(descDatabase.getDatabaseName());
 
       if (database == null) {
@@ -2851,9 +2835,7 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           database.getDescription(), location,
           database.getOwnerName(), (null == ownerType) ? null : ownerType.name(), params);
 
-      outStream.close();
-      outStream = null;
-    } catch (IOException e) {
+    } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR);
     } finally {
       IOUtils.closeStream(outStream);
@@ -2899,16 +2881,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
     }
 
     // write the results in the file
-    DataOutputStream outStream = null;
+    DataOutputStream outStream = getOutputStream(showTblStatus.getResFile());
     try {
-      Path resFile = new Path(showTblStatus.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
-
       formatter.showTableStatus(outStream, db, conf, tbls, part, par);
-
-      outStream.close();
-      outStream = null;
     } catch (Exception e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, "show table status");
     } finally {
@@ -3011,40 +2986,22 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
 
     // describe the table - populate the output stream
     Table tbl = db.getTable(tableName, false);
+    if (tbl == null) {
+      throw new HiveException(ErrorMsg.INVALID_TABLE, tableName);
+    }
     Partition part = null;
-    DataOutputStream outStream = null;
-    try {
-      Path resFile = new Path(descTbl.getResFile());
-      if (tbl == null) {
-        FileSystem fs = resFile.getFileSystem(conf);
-        outStream = fs.create(resFile);
-        outStream.close();
-        outStream = null;
-        throw new HiveException(ErrorMsg.INVALID_TABLE, tableName);
-      }
-      if (descTbl.getPartSpec() != null) {
-        part = db.getPartition(tbl, descTbl.getPartSpec(), false);
-        if (part == null) {
-          FileSystem fs = resFile.getFileSystem(conf);
-          outStream = fs.create(resFile);
-          outStream.close();
-          outStream = null;
-          throw new HiveException(ErrorMsg.INVALID_PARTITION,
-              StringUtils.join(descTbl.getPartSpec().keySet(), ','), tableName);
-        }
-        tbl = part.getTable();
+    if (descTbl.getPartSpec() != null) {
+      part = db.getPartition(tbl, descTbl.getPartSpec(), false);
+      if (part == null) {
+        throw new HiveException(ErrorMsg.INVALID_PARTITION,
+            StringUtils.join(descTbl.getPartSpec().keySet(), ','), tableName);
       }
-    } catch (IOException e) {
-      throw new HiveException(e, ErrorMsg.GENERIC_ERROR, tableName);
-    } finally {
-      IOUtils.closeStream(outStream);
+      tbl = part.getTable();
     }
 
+    DataOutputStream outStream = getOutputStream(descTbl.getResFile());
     try {
       LOG.info("DDLTask: got data for " + tbl.getTableName());
-      Path resFile = new Path(descTbl.getResFile());
-      FileSystem fs = resFile.getFileSystem(conf);
-      outStream = fs.create(resFile);
 
       List<FieldSchema> cols = null;
       List<ColumnStatisticsObj> colStats = null;
@@ -3092,13 +3049,9 @@ public class DDLTask extends Task<DDLWork> implements Serializable {
           descTbl.isPretty(), isOutputPadded, colStats);
 
       LOG.info("DDLTask: written data for " + tbl.getTableName());
-      outStream.close();
-      outStream = null;
 
     } catch (SQLException e) {
       throw new HiveException(e, ErrorMsg.GENERIC_ERROR, tableName);
-    } catch (IOException e) {
-      throw new HiveException(e, ErrorMsg.GENERIC_ERROR, tableName);
     } finally {
       IOUtils.closeStream(outStream);
     }

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
index 9f8c756..2d7d9d4 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
@@ -106,6 +106,7 @@ import org.apache.hadoop.hive.ql.plan.RoleDDLDesc;
 import org.apache.hadoop.hive.ql.plan.ShowColumnsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCompactionsDesc;
 import org.apache.hadoop.hive.ql.plan.ShowConfDesc;
+import org.apache.hadoop.hive.ql.plan.ShowCreateDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -413,6 +414,10 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowPartitions(ast);
       break;
+    case HiveParser.TOK_SHOW_CREATEDATABASE:
+      ctx.setResFile(ctx.getLocalTmpPath());
+      analyzeShowCreateDatabase(ast);
+      break;
     case HiveParser.TOK_SHOW_CREATETABLE:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowCreateTable(ast);
@@ -2078,6 +2083,18 @@ public class DDLSemanticAnalyzer extends BaseSemanticAnalyzer {
     setFetchTask(createFetchTask(showPartsDesc.getSchema()));
   }
 
+  private void analyzeShowCreateDatabase(ASTNode ast) throws SemanticException {
+    String dbName = getUnescapedName((ASTNode)ast.getChild(0));
+    ShowCreateDatabaseDesc showCreateDbDesc =
+        new ShowCreateDatabaseDesc(dbName, ctx.getResFile().toString());
+
+    Database database = getDatabase(dbName);
+    inputs.add(new ReadEntity(database));
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
+        showCreateDbDesc), conf));
+    setFetchTask(createFetchTask(showCreateDbDesc.getSchema()));
+  }
+
   private void analyzeShowCreateTable(ASTNode ast) throws SemanticException {
     ShowCreateTableDesc showCreateTblDesc;
     String tableName = getUnescapedName((ASTNode)ast.getChild(0));

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
index 3969a54..3df67e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
@@ -178,6 +178,7 @@ TOK_SHOWTABLES;
 TOK_SHOWCOLUMNS;
 TOK_SHOWFUNCTIONS;
 TOK_SHOWPARTITIONS;
+TOK_SHOW_CREATEDATABASE;
 TOK_SHOW_CREATETABLE;
 TOK_SHOW_TABLESTATUS;
 TOK_SHOW_TBLPROPERTIES;
@@ -1374,7 +1375,11 @@ showStatement
     -> ^(TOK_SHOWCOLUMNS tableName $db_name?)
     | KW_SHOW KW_FUNCTIONS (KW_LIKE showFunctionIdentifier|showFunctionIdentifier)?  -> ^(TOK_SHOWFUNCTIONS KW_LIKE? showFunctionIdentifier?)
     | KW_SHOW KW_PARTITIONS tabName=tableName partitionSpec? -> ^(TOK_SHOWPARTITIONS $tabName partitionSpec?) 
-    | KW_SHOW KW_CREATE KW_TABLE tabName=tableName -> ^(TOK_SHOW_CREATETABLE $tabName)
+    | KW_SHOW KW_CREATE (
+        (KW_DATABASE|KW_SCHEMA) => (KW_DATABASE|KW_SCHEMA) db_name=identifier -> ^(TOK_SHOW_CREATEDATABASE $db_name)
+        |
+        KW_TABLE tabName=tableName -> ^(TOK_SHOW_CREATETABLE $tabName)
+      )
     | KW_SHOW KW_TABLE KW_EXTENDED ((KW_FROM|KW_IN) db_name=identifier)? KW_LIKE showStmtIdentifier partitionSpec?
     -> ^(TOK_SHOW_TABLESTATUS showStmtIdentifier $db_name? partitionSpec?)
     | KW_SHOW KW_TBLPROPERTIES tableName (LPAREN prptyName=StringLiteral RPAREN)? -> ^(TOK_SHOW_TBLPROPERTIES tableName $prptyName?)

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
index a2fbc11..0affe84 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
@@ -65,6 +65,7 @@ public final class SemanticAnalyzerFactory {
     commandType.put(HiveParser.TOK_SHOWCOLUMNS, HiveOperation.SHOWCOLUMNS);
     commandType.put(HiveParser.TOK_SHOW_TABLESTATUS, HiveOperation.SHOW_TABLESTATUS);
     commandType.put(HiveParser.TOK_SHOW_TBLPROPERTIES, HiveOperation.SHOW_TBLPROPERTIES);
+    commandType.put(HiveParser.TOK_SHOW_CREATEDATABASE, HiveOperation.SHOW_CREATEDATABASE);
     commandType.put(HiveParser.TOK_SHOW_CREATETABLE, HiveOperation.SHOW_CREATETABLE);
     commandType.put(HiveParser.TOK_SHOWFUNCTIONS, HiveOperation.SHOWFUNCTIONS);
     commandType.put(HiveParser.TOK_SHOWINDEXES, HiveOperation.SHOWINDEXES);
@@ -227,6 +228,7 @@ public final class SemanticAnalyzerFactory {
       case HiveParser.TOK_SHOWCOLUMNS:
       case HiveParser.TOK_SHOW_TABLESTATUS:
       case HiveParser.TOK_SHOW_TBLPROPERTIES:
+      case HiveParser.TOK_SHOW_CREATEDATABASE:
       case HiveParser.TOK_SHOW_CREATETABLE:
       case HiveParser.TOK_SHOWFUNCTIONS:
       case HiveParser.TOK_SHOWPARTITIONS:

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
index 8dbb3c1..a4c3db1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
@@ -57,6 +57,7 @@ public class DDLWork implements Serializable {
   private ShowTxnsDesc showTxnsDesc;
   private DescFunctionDesc descFunctionDesc;
   private ShowPartitionsDesc showPartsDesc;
+  private ShowCreateDatabaseDesc showCreateDbDesc;
   private ShowCreateTableDesc showCreateTblDesc;
   private DescTableDesc descTblDesc;
   private AddPartitionDesc addPartitionDesc;
@@ -367,6 +368,16 @@ public class DDLWork implements Serializable {
   }
 
   /**
+   * @param showCreateDbDesc
+   */
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+      ShowCreateDatabaseDesc showCreateDbDesc) {
+    this(inputs, outputs);
+
+    this.showCreateDbDesc = showCreateDbDesc;
+  }
+
+  /**
    * @param showCreateTblDesc
    */
   public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -832,6 +843,16 @@ public class DDLWork implements Serializable {
     this.showPartsDesc = showPartsDesc;
   }
 
+  @Explain(displayName = "Show Create Database Operator",
+      explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public ShowCreateDatabaseDesc getShowCreateDbDesc() {
+    return showCreateDbDesc;
+  }
+
+  public void setShowCreateDbDesc(ShowCreateDatabaseDesc showCreateDbDesc) {
+    this.showCreateDbDesc = showCreateDbDesc;
+  }
+
   /**
    * @return the showCreateTblDesc
    */

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
index dee2136..af7e43e 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
@@ -64,6 +64,7 @@ public enum HiveOperation {
   SHOWCOLUMNS("SHOWCOLUMNS", null, null),
   SHOW_TABLESTATUS("SHOW_TABLESTATUS", null, null),
   SHOW_TBLPROPERTIES("SHOW_TBLPROPERTIES", null, null),
+  SHOW_CREATEDATABASE("SHOW_CREATEDATABASE", new Privilege[]{Privilege.SELECT}, null),
   SHOW_CREATETABLE("SHOW_CREATETABLE", new Privilege[]{Privilege.SELECT}, null),
   SHOWFUNCTIONS("SHOWFUNCTIONS", null, null),
   SHOWINDEXES("SHOWINDEXES", null, null),

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCreateDatabaseDesc.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCreateDatabaseDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCreateDatabaseDesc.java
new file mode 100644
index 0000000..2b12691
--- /dev/null
+++ b/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCreateDatabaseDesc.java
@@ -0,0 +1,94 @@
+/**
+ * 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.hadoop.hive.ql.plan;
+
+import java.io.Serializable;
+
+import org.apache.hadoop.hive.ql.plan.Explain.Level;
+
+
+/**
+ * ShowCreateDatabaseDesc.
+ *
+ */
+@Explain(displayName = "Show Create Database",
+    explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+public class ShowCreateDatabaseDesc extends DDLDesc implements Serializable {
+  private static final long serialVersionUID = 1L;
+  String resFile;
+  String dbName;
+
+  /**
+   * thrift ddl for the result of showcreatedatabase.
+   */
+  private static final String schema = "createdb_stmt#string";
+
+  public String getSchema() {
+    return schema;
+  }
+
+  /**
+   * For serialization use only.
+   */
+  public ShowCreateDatabaseDesc() {
+  }
+
+  /**
+   * @param resFile
+   * @param dbName
+   *          name of database to show
+   */
+  public ShowCreateDatabaseDesc(String dbName, String resFile) {
+    this.dbName = dbName;
+    this.resFile = resFile;
+  }
+
+  /**
+   * @return the resFile
+   */
+  @Explain(displayName = "result file", explainLevels = { Level.EXTENDED })
+  public String getResFile() {
+    return resFile;
+  }
+
+  /**
+   * @param resFile
+   *          the resFile to set
+   */
+  public void setResFile(String resFile) {
+    this.resFile = resFile;
+  }
+
+  /**
+   * @return the databaseName
+   */
+  @Explain(displayName = "database name",
+      explainLevels = { Level.USER, Level.DEFAULT, Level.EXTENDED })
+  public String getDatabaseName() {
+    return dbName;
+  }
+
+  /**
+   * @param databaseName
+   *          the dbName to set
+   */
+  public void setDatabaseName(String dbName) {
+    this.dbName = dbName;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
index 71be469..5418e9a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
@@ -66,6 +66,7 @@ public enum HiveOperationType {
   SHOWCOLUMNS,
   SHOW_TABLESTATUS,
   SHOW_TBLPROPERTIES,
+  SHOW_CREATEDATABASE,
   SHOW_CREATETABLE,
   SHOWFUNCTIONS,
   SHOWINDEXES,

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
----------------------------------------------------------------------
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
index 8e61d57..ca8f53f 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
@@ -309,6 +309,8 @@ public class Operation2Privilege {
     // for now require select WITH GRANT
     op2Priv.put(HiveOperationType.SHOW_CREATETABLE, PrivRequirement.newIOPrivRequirement
 (SEL_GRANT_AR, null));
+    op2Priv.put(HiveOperationType.SHOW_CREATEDATABASE, PrivRequirement.newIOPrivRequirement
+(SEL_GRANT_AR, null));
 
     // for now allow only create-view with 'select with grant'
     // the owner will also have select with grant privileges on new view

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/test/queries/clientpositive/show_create_database.q
----------------------------------------------------------------------
diff --git a/ql/src/test/queries/clientpositive/show_create_database.q b/ql/src/test/queries/clientpositive/show_create_database.q
new file mode 100644
index 0000000..6136f23
--- /dev/null
+++ b/ql/src/test/queries/clientpositive/show_create_database.q
@@ -0,0 +1,3 @@
+CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue');
+SHOW CREATE DATABASE some_database;
+

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/test/results/clientpositive/show_create_database.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/show_create_database.q.out b/ql/src/test/results/clientpositive/show_create_database.q.out
new file mode 100644
index 0000000..4755d2d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/show_create_database.q.out
@@ -0,0 +1,19 @@
+PREHOOK: query: CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:some_database
+POSTHOOK: query: CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:some_database
+PREHOOK: query: SHOW CREATE DATABASE some_database
+PREHOOK: type: SHOW_CREATEDATABASE
+PREHOOK: Input: database:some_database
+POSTHOOK: query: SHOW CREATE DATABASE some_database
+POSTHOOK: type: SHOW_CREATEDATABASE
+POSTHOOK: Input: database:some_database
+CREATE DATABASE `some_database`
+COMMENT
+  'for show create db test'
+LOCATION
+#### A masked pattern was here ####
+WITH DBPROPERTIES (
+  'somekey'='somevalue')

http://git-wip-us.apache.org/repos/asf/hive/blob/3cf7bd9e/ql/src/test/results/clientpositive/tez/show_create_database.q.out
----------------------------------------------------------------------
diff --git a/ql/src/test/results/clientpositive/tez/show_create_database.q.out b/ql/src/test/results/clientpositive/tez/show_create_database.q.out
new file mode 100644
index 0000000..4755d2d
--- /dev/null
+++ b/ql/src/test/results/clientpositive/tez/show_create_database.q.out
@@ -0,0 +1,19 @@
+PREHOOK: query: CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')
+PREHOOK: type: CREATEDATABASE
+PREHOOK: Output: database:some_database
+POSTHOOK: query: CREATE DATABASE some_database comment 'for show create db test' WITH DBPROPERTIES ('somekey'='somevalue')
+POSTHOOK: type: CREATEDATABASE
+POSTHOOK: Output: database:some_database
+PREHOOK: query: SHOW CREATE DATABASE some_database
+PREHOOK: type: SHOW_CREATEDATABASE
+PREHOOK: Input: database:some_database
+POSTHOOK: query: SHOW CREATE DATABASE some_database
+POSTHOOK: type: SHOW_CREATEDATABASE
+POSTHOOK: Input: database:some_database
+CREATE DATABASE `some_database`
+COMMENT
+  'for show create db test'
+LOCATION
+#### A masked pattern was here ####
+WITH DBPROPERTIES (
+  'somekey'='somevalue')