You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by mg...@apache.org on 2019/07/25 09:58:07 UTC

[hive] branch master updated: HIVE-22010 - Clean up ShowCreateTableOperation

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 68ae4a5  HIVE-22010 - Clean up ShowCreateTableOperation
68ae4a5 is described below

commit 68ae4a5cd1b916098dc1deb2bcede5f862afd80e
Author: miklosgergely <mg...@cloudera.com>
AuthorDate: Fri Jul 19 15:20:14 2019 +0200

    HIVE-22010 - Clean up ShowCreateTableOperation
---
 .../hive/ql/parse/TestReplicationScenarios.java    |  31 +-
 .../org/apache/hadoop/hive/ql/ddl/DDLUtils.java    |   9 +-
 .../org/apache/hadoop/hive/ql/ddl/DDLWork.java     |  11 +-
 .../AlterDatabaseSetPropertiesOperation.java       |   2 +-
 .../ql/ddl/database/CreateDatabaseOperation.java   |   4 +-
 .../hive/ql/ddl/function/ShowFunctionsDesc.java    |   6 +-
 .../hadoop/hive/ql/ddl/misc/MsckOperation.java     |   4 +-
 .../ql/ddl/process/ShowCompactionsOperation.java   |   2 +-
 .../ql/ddl/process/ShowTransactionsOperation.java  |   2 +-
 .../ql/ddl/table/AbstractAlterTableOperation.java  |   2 +-
 .../ql/ddl/table/creation/CreateTableDesc.java     |   8 +-
 .../hive/ql/ddl/table/creation/DropTableDesc.java  |   4 +-
 .../table/creation/ShowCreateTableOperation.java   | 365 +++++++++++----------
 .../hive/ql/ddl/table/info/DescTableOperation.java |   2 +-
 .../hive/ql/ddl/table/lock/ShowLocksOperation.java |   2 +-
 .../misc/AlterTableSetPropertiesOperation.java     |   5 +-
 .../partition/AlterTableAddPartitionDesc.java      |   4 +-
 .../partition/AlterTableDropPartitionDesc.java     |   2 +-
 .../AlterTableDropPartitionOperation.java          |   4 +-
 .../table/storage/AlterTableArchiveOperation.java  |   2 +-
 .../storage/AlterTableUnarchiveOperation.java      |   2 +-
 .../hadoop/hive/ql/ddl/view/CreateViewDesc.java    |   4 +-
 .../hive/ql/ddl/view/CreateViewOperation.java      |   2 +-
 .../clientpositive/show_create_table_alter.q.out   |   1 -
 24 files changed, 234 insertions(+), 246 deletions(-)

diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
index 7014baa..ce70952 100644
--- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
+++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/parse/TestReplicationScenarios.java
@@ -186,6 +186,8 @@ public class TestReplicationScenarios {
     });
 
     MetaStoreTestUtils.startMetaStoreWithRetry(hconf);
+    // re set the WAREHOUSE property to the test dir, as the previous command added a random port to it
+    hconf.set(MetastoreConf.ConfVars.WAREHOUSE.getVarname(), System.getProperty("test.warehouse.dir", "/tmp"));
 
     Path testPath = new Path(TEST_PATH);
     FileSystem fs = FileSystem.get(testPath.toUri(),hconf);
@@ -459,7 +461,6 @@ public class TestReplicationScenarios {
     String unptn_locn = new Path(TEST_PATH, name + "_unptn").toUri().getPath();
     String ptn_locn_1 = new Path(TEST_PATH, name + "_ptn1").toUri().getPath();
     String ptn_locn_2 = new Path(TEST_PATH, name + "_ptn2").toUri().getPath();
-    String ptn_locn_2_later = new Path(TEST_PATH, name + "_ptn2_later").toUri().getPath();
 
     createTestDataFile(unptn_locn, unptn_data);
     createTestDataFile(ptn_locn_1, ptn_data_1);
@@ -549,7 +550,6 @@ public class TestReplicationScenarios {
     String[] unptn_data = new String[]{ "eleven" , "twelve" };
     String[] ptn_data_1 = new String[]{ "thirteen", "fourteen", "fifteen"};
     String[] ptn_data_2 = new String[]{ "fifteen", "sixteen", "seventeen"};
-    String[] empty = new String[]{};
 
     String unptn_locn = new Path(TEST_PATH, name + "_unptn").toUri().getPath();
     String ptn_locn_1 = new Path(TEST_PATH, name + "_ptn1").toUri().getPath();
@@ -694,7 +694,6 @@ public class TestReplicationScenarios {
     run("CREATE TABLE " + dbName + ".ptned(a string) partitioned by (b int) STORED AS TEXTFILE", driver);
 
     String[] ptn_data = new String[]{ "eleven" , "twelve" };
-    String[] empty = new String[]{};
     String ptn_locn = new Path(TEST_PATH, name + "_ptn").toUri().getPath();
 
     createTestDataFile(ptn_locn, ptn_data);
@@ -767,7 +766,6 @@ public class TestReplicationScenarios {
     run("CREATE TABLE " + dbName + ".ptned(a string) partitioned by (b int) STORED AS TEXTFILE", driver);
 
     String[] ptn_data = new String[]{ "eleven" , "twelve" };
-    String[] empty = new String[]{};
     String ptn_locn = new Path(TEST_PATH, name + "_ptn").toUri().getPath();
 
     createTestDataFile(ptn_locn, ptn_data);
@@ -2572,7 +2570,6 @@ public class TestReplicationScenarios {
     // List to maintain the incremental dumps for each operation
     List<Tuple> incrementalDumpList = new ArrayList<Tuple>();
 
-    String[] empty = new String[] {};
     String[] unptn_data = new String[] { "ten" };
     String[] ptn_data_1 = new String[] { "fifteen" };
     String[] ptn_data_2 = new String[] { "seventeen" };
@@ -2682,7 +2679,6 @@ public class TestReplicationScenarios {
     run("CREATE TABLE " + dbName + ".unptned(a string) STORED AS ORC", driver);
 
     String[] unptn_data = new String[] { "eleven", "twelve" };
-    String[] empty = new String[] {};
     run("INSERT INTO TABLE " + dbName + ".unptned values('" + unptn_data[0] + "')", driver);
 
     // Bootstrap dump/load
@@ -2695,7 +2691,7 @@ public class TestReplicationScenarios {
     verifyRun("SELECT a from " + dbName + ".unptned ORDER BY a", unptn_data, driver);
 
     // Replicate all the events happened after bootstrap
-    Tuple incrDump = incrementalLoadAndVerify(dbName, bootstrapDump.lastReplId, replDbName);
+    incrementalLoadAndVerify(dbName, bootstrapDump.lastReplId, replDbName);
 
     // migration test is failing as CONCATENATE is not working. Its not creating the merged file.
     if (!isMigrationTest) {
@@ -2728,7 +2724,7 @@ public class TestReplicationScenarios {
     run("ALTER TABLE " + dbName + ".ptned PARTITION(b=2) CONCATENATE", driver);
 
     // Replicate all the events happened so far
-    Tuple incrDump = incrementalLoadAndVerify(dbName, bootstrapDump.lastReplId, replDbName);
+    incrementalLoadAndVerify(dbName, bootstrapDump.lastReplId, replDbName);
 
     // migration test is failing as CONCATENATE is not working. Its not creating the merged file.
     if (!isMigrationTest) {
@@ -3268,7 +3264,7 @@ public class TestReplicationScenarios {
     run("CREATE TABLE " + dbName + ".normal(a int)", driver);
     run("INSERT INTO " + dbName + ".normal values (1)", driver);
 
-    Path path = new Path(System.getProperty("test.warehouse.dir", ""));
+    Path path = new Path(System.getProperty("test.warehouse.dir", "/tmp"));
     path = new Path(path, dbName.toLowerCase() + ".db");
     path = new Path(path, "normal");
     FileSystem fs = path.getFileSystem(hconf);
@@ -3288,7 +3284,7 @@ public class TestReplicationScenarios {
     run("CREATE TABLE " + dbName + ".normal(a int) PARTITIONED BY (part int)", driver);
     run("INSERT INTO " + dbName + ".normal partition (part= 124) values (1)", driver);
 
-    Path path = new Path(System.getProperty("test.warehouse.dir",""));
+    Path path = new Path(System.getProperty("test.warehouse.dir", "/tmp"));
     path = new Path(path, dbName.toLowerCase()+".db");
     path = new Path(path, "normal");
     path = new Path(path, "part=124");
@@ -3546,12 +3542,6 @@ public class TestReplicationScenarios {
     return results;
   }
 
-  private void printOutput(IDriver myDriver) throws IOException {
-    for (String s : getOutput(myDriver)){
-      LOG.info(s);
-    }
-  }
-
   private void verifyIfTableNotExist(String dbName, String tableName, HiveMetaStoreClient myClient){
     Exception e = null;
     try {
@@ -3594,15 +3584,6 @@ public class TestReplicationScenarios {
     }
   }
 
-  private void verifyIfDirNotExist(FileSystem fs, Path path, PathFilter filter){
-    try {
-      FileStatus[] statuses = fs.listStatus(path, filter);
-      assertEquals(0, statuses.length);
-    } catch (IOException e) {
-      assert(false);
-    }
-  }
-
   private void verifySetup(String cmd, String[] data, IDriver myDriver) throws  IOException {
     if (VERIFY_SETUP_STEPS){
       run(cmd, myDriver);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
index c3d5f90..c81c574 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLUtils.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.TreeMap;
 
 import org.apache.commons.lang.StringUtils;
@@ -84,8 +85,8 @@ public final class DDLUtils {
    * @return {@code true} if item was added
    */
   public static boolean addIfAbsentByName(WriteEntity newWriteEntity, Set<WriteEntity> outputs) {
-    for(WriteEntity writeEntity : outputs) {
-      if(writeEntity.getName().equalsIgnoreCase(newWriteEntity.getName())) {
+    for (WriteEntity writeEntity : outputs) {
+      if (writeEntity.getName().equalsIgnoreCase(newWriteEntity.getName())) {
         LOG.debug("Ignoring request to add {} because {} is present", newWriteEntity.toStringDetail(),
             writeEntity.toStringDetail());
         return false;
@@ -153,12 +154,12 @@ public final class DDLUtils {
     return false;
   }
 
-  public static String propertiesToString(Map<String, String> props, List<String> exclude) {
+  public static String propertiesToString(Map<String, String> props, Set<String> exclude) {
     if (props.isEmpty()) {
       return "";
     }
 
-    Map<String, String> sortedProperties = new TreeMap<String, String>(props);
+    SortedMap<String, String> sortedProperties = new TreeMap<String, String>(props);
     List<String> realProps = new ArrayList<String>();
     for (Map.Entry<String, String> e : sortedProperties.entrySet()) {
       if (e.getValue() != null && (exclude == null || !exclude.contains(e.getKey()))) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLWork.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLWork.java
index a7c6c12..6eea86b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLWork.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/DDLWork.java
@@ -33,23 +33,16 @@ public final class DDLWork implements Serializable {
   private static final long serialVersionUID = 1L;
 
   private DDLDesc ddlDesc;
-  boolean needLock = false;
+  private boolean needLock = false;
 
   /** ReadEntitites that are passed to the hooks. */
   private Set<ReadEntity> inputs;
   /** List of WriteEntities that are passed to the hooks. */
   private Set<WriteEntity> outputs;
 
-  public DDLWork() {
-  }
-
-  public DDLWork(Set<ReadEntity> inputs, Set<WriteEntity> outputs) {
+  public DDLWork(Set<ReadEntity> inputs, Set<WriteEntity> outputs, DDLDesc ddlDesc) {
     this.inputs = inputs;
     this.outputs = outputs;
-  }
-
-  public DDLWork(Set<ReadEntity> inputs, Set<WriteEntity> outputs, DDLDesc ddlDesc) {
-    this(inputs, outputs);
     this.ddlDesc = ddlDesc;
   }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java
index 7f73502..12ec9e9 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/AlterDatabaseSetPropertiesOperation.java
@@ -27,7 +27,7 @@ import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
  * Operation process of altering a database's properties.
  */
 public class AlterDatabaseSetPropertiesOperation
-  extends AbstractAlterDatabaseOperation<AlterDatabaseSetPropertiesDesc> {
+    extends AbstractAlterDatabaseOperation<AlterDatabaseSetPropertiesDesc> {
   public AlterDatabaseSetPropertiesOperation(DDLOperationContext context, AlterDatabaseSetPropertiesDesc desc) {
     super(context, desc);
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/CreateDatabaseOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/CreateDatabaseOperation.java
index a7eabe7..bc31974 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/CreateDatabaseOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/database/CreateDatabaseOperation.java
@@ -19,10 +19,10 @@
 package org.apache.hadoop.hive.ql.ddl.database;
 
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.PrincipalType;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.ErrorMsg;
 import org.apache.hadoop.hive.ql.ddl.DDLOperation;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
@@ -66,7 +66,7 @@ public class CreateDatabaseOperation extends DDLOperation<CreateDatabaseDesc> {
       database.setLocationUri(Utilities.getQualifiedPath(context.getConf(), new Path(database.getLocationUri())));
     } else {
       // Location is not set we utilize METASTOREWAREHOUSE together with database name
-      Path path = new Path(HiveConf.getVar(context.getConf(), HiveConf.ConfVars.METASTOREWAREHOUSE),
+      Path path = new Path(MetastoreConf.getVar(context.getConf(), MetastoreConf.ConfVars.WAREHOUSE),
           database.getName().toLowerCase() + DATABASE_PATH_SUFFIX);
       String qualifiedPath = Utilities.getQualifiedPath(context.getConf(), path);
       database.setLocationUri(qualifiedPath);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/ShowFunctionsDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/ShowFunctionsDesc.java
index 08977e1..9784b1a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/ShowFunctionsDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/function/ShowFunctionsDesc.java
@@ -38,14 +38,10 @@ public class ShowFunctionsDesc implements DDLDesc, Serializable {
   private final String pattern;
 
   public ShowFunctionsDesc(Path resFile) {
-    this(resFile, null, false);
+    this(resFile, null);
   }
 
   public ShowFunctionsDesc(Path resFile, String pattern) {
-    this(resFile, pattern, false);
-  }
-
-  public ShowFunctionsDesc(Path resFile, String pattern, boolean isLikePattern) {
     this.resFile = resFile.toString();
     this.pattern = pattern;
   }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/MsckOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/MsckOperation.java
index ab8cf46..e027f4a 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/MsckOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/misc/MsckOperation.java
@@ -60,8 +60,8 @@ public class MsckOperation extends DDLOperation<MsckDesc> {
       try (HiveMetaStoreClient msc = new HiveMetaStoreClient(context.getConf())) {
         Table table = msc.getTable(SessionState.get().getCurrentCatalog(), names[0], names[1]);
         String qualifiedTableName = Warehouse.getCatalogQualifiedTableName(table);
-        boolean msckEnablePartitionRetention = context.getConf().getBoolean(
-            MetastoreConf.ConfVars.MSCK_REPAIR_ENABLE_PARTITION_RETENTION.getHiveName(), false);
+        boolean msckEnablePartitionRetention = MetastoreConf.getBoolVar(context.getConf(),
+            MetastoreConf.ConfVars.MSCK_REPAIR_ENABLE_PARTITION_RETENTION);
         if (msckEnablePartitionRetention) {
           partitionExpirySeconds = PartitionManagementTask.getRetentionPeriodInSeconds(table);
           LOG.info("{} - Retention period ({}s) for partition is enabled for MSCK REPAIR..", qualifiedTableName,
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowCompactionsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowCompactionsOperation.java
index 4bf45fc..b04404d 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowCompactionsOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowCompactionsOperation.java
@@ -48,7 +48,7 @@ public class ShowCompactionsOperation extends DDLOperation<ShowCompactionsDesc>
     // Write the results into the file
     try (DataOutputStream os = DDLUtils.getOutputStream(new Path(desc.getResFile()), context)) {
       // Write a header for cliDriver
-      if(!sessionState.isHiveServerQuery()) {
+      if (!sessionState.isHiveServerQuery()) {
         writeHeader(os);
       }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowTransactionsOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowTransactionsOperation.java
index 805272e..6f28855 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowTransactionsOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/process/ShowTransactionsOperation.java
@@ -48,7 +48,7 @@ public class ShowTransactionsOperation extends DDLOperation<ShowTransactionsDesc
 
     // Write the results into the file
     try (DataOutputStream os = DDLUtils.getOutputStream(new Path(desc.getResFile()), context)) {
-      if(!sessionState.isHiveServerQuery()) {
+      if (!sessionState.isHiveServerQuery()) {
         writeHeader(os);
       }
 
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableOperation.java
index 8c40fab..e1f9fad 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/AbstractAlterTableOperation.java
@@ -124,7 +124,7 @@ public abstract class AbstractAlterTableOperation<T extends AbstractAlterTableDe
     return (part == null ? tbl.getTTable().getSd() : part.getTPartition().getSd());
   }
 
-  public void finalizeAlterTableWithWriteIdOp(Table table, Table oldTable, List<Partition> partitions,
+  private void finalizeAlterTableWithWriteIdOp(Table table, Table oldTable, List<Partition> partitions,
       DDLOperationContext context, EnvironmentContext environmentContext, AbstractAlterTableDesc alterTable)
       throws HiveException {
     if (partitions == null) {
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/CreateTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/CreateTableDesc.java
index f713b22..1b6db58 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/CreateTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/CreateTableDesc.java
@@ -247,7 +247,7 @@ public class CreateTableDesc implements DDLDesc, Serializable {
     return tableName;
   }
 
-  public String getDatabaseName(){
+  public String getDatabaseName() {
     return databaseName;
   }
 
@@ -606,7 +606,7 @@ public class CreateTableDesc implements DDLDesc, Serializable {
         } catch (Exception err) {
           LOG.error("Failed to get type info", err);
         }
-        if(null == pti){
+        if (null == pti) {
           throw new SemanticException(ErrorMsg.PARTITION_COLUMN_NON_PRIMITIVE.getMsg() + " Found "
               + partCol + " of type: " + fs.getType());
         }
@@ -711,8 +711,8 @@ public class CreateTableDesc implements DDLDesc, Serializable {
    * @return what kind of replication scope this drop is running under.
    * This can result in a "CREATE/REPLACE IF NEWER THAN" kind of semantic
    */
-  public ReplicationSpec getReplicationSpec(){
-    if (replicationSpec == null){
+  public ReplicationSpec getReplicationSpec() {
+    if (replicationSpec == null) {
       this.replicationSpec = new ReplicationSpec();
     }
     return this.replicationSpec;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
index e1a1fab..1410d60 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/DropTableDesc.java
@@ -68,11 +68,11 @@ public class DropTableDesc implements DDLDesc, Serializable {
    * @return what kind of replication scope this drop is running under.
    * This can result in a "DROP IF OLDER THAN" kind of semantic
    */
-  public ReplicationSpec getReplicationSpec(){
+  public ReplicationSpec getReplicationSpec() {
     return replicationSpec;
   }
 
-  public boolean getValidationRequired(){
+  public boolean getValidationRequired() {
     return validationRequired;
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/ShowCreateTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/ShowCreateTableOperation.java
index 6a3f00d..ac54138 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/ShowCreateTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/creation/ShowCreateTableOperation.java
@@ -27,10 +27,13 @@ import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.TreeMap;
 import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
 
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.fs.Path;
@@ -50,20 +53,25 @@ import org.apache.hadoop.hive.serde.serdeConstants;
 import org.apache.hive.common.util.HiveStringUtils;
 import org.stringtemplate.v4.ST;
 
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+
 /**
  * Operation process showing the creation of a table.
  */
 public class ShowCreateTableOperation extends DDLOperation<ShowCreateTableDesc> {
   private static final String EXTERNAL = "external";
   private static final String TEMPORARY = "temporary";
+  private static final String NAME = "name";
   private static final String LIST_COLUMNS = "columns";
-  private static final String TBL_COMMENT = "tbl_comment";
-  private static final String LIST_PARTITIONS = "partitions";
-  private static final String SORT_BUCKET = "sort_bucket";
-  private static final String SKEWED_INFO = "tbl_skewedinfo";
+  private static final String COMMENT = "comment";
+  private static final String PARTITIONS = "partitions";
+  private static final String BUCKETS = "buckets";
+  private static final String SKEWED = "skewedinfo";
   private static final String ROW_FORMAT = "row_format";
-  private static final String TBL_LOCATION = "tbl_location";
-  private static final String TBL_PROPERTIES = "tbl_properties";
+  private static final String LOCATION_BLOCK = "location_block";
+  private static final String LOCATION = "location";
+  private static final String PROPERTIES = "properties";
 
   public ShowCreateTableOperation(DDLOperationContext context, ShowCreateTableDesc desc) {
     super(context, desc);
@@ -73,198 +81,207 @@ public class ShowCreateTableOperation extends DDLOperation<ShowCreateTableDesc>
   public int execute() throws HiveException {
     // get the create table statement for the table and populate the output
     try (DataOutputStream outStream = DDLUtils.getOutputStream(new Path(desc.getResFile()), context)) {
-      return showCreateTable(outStream);
+      Table table = context.getDb().getTable(desc.getTableName(), false);
+      String command = table.isView() ?
+          getCreateViewCommand(table) :
+          getCreateTableCommand(table);
+      outStream.write(command.getBytes(StandardCharsets.UTF_8));
+      return 0;
+    } catch (IOException e) {
+      LOG.info("show create table: ", e);
+      return 1;
     } catch (Exception e) {
       throw new HiveException(e);
     }
   }
 
-  private int showCreateTable(DataOutputStream outStream) throws HiveException {
-    boolean needsLocation = true;
-    StringBuilder createTabCommand = new StringBuilder();
+  private static final String CREATE_VIEW_COMMAND = "CREATE VIEW `%s` AS %s";
 
-    Table tbl = context.getDb().getTable(desc.getTableName(), false);
-    List<String> duplicateProps = new ArrayList<String>();
-    try {
-      needsLocation = CreateTableOperation.doesTableNeedLocation(tbl);
+  private String getCreateViewCommand(Table table) {
+    return String.format(CREATE_VIEW_COMMAND, desc.getTableName(), table.getViewExpandedText());
+  }
 
-      if (tbl.isView()) {
-        String createTabStmt = "CREATE VIEW `" + desc.getTableName() + "` AS " + tbl.getViewExpandedText();
-        outStream.write(createTabStmt.getBytes(StandardCharsets.UTF_8));
-        return 0;
-      }
+  private static final String CREATE_TABLE_TEMPLATE =
+      "CREATE <" + TEMPORARY + "><" + EXTERNAL + ">TABLE `<" + NAME + ">`(\n" +
+      "<" + LIST_COLUMNS + ">)\n" +
+      "<" + COMMENT + ">\n" +
+      "<" + PARTITIONS + ">\n" +
+      "<" + BUCKETS + ">\n" +
+      "<" + SKEWED + ">\n" +
+      "<" + ROW_FORMAT + ">\n" +
+      "<" + LOCATION_BLOCK + ">" +
+      "TBLPROPERTIES (\n" +
+      "<" + PROPERTIES + ">)\n";
 
-      createTabCommand.append("CREATE <" + TEMPORARY + "><" + EXTERNAL + ">TABLE `");
-      createTabCommand.append(desc.getTableName() + "`(\n");
-      createTabCommand.append("<" + LIST_COLUMNS + ">)\n");
-      createTabCommand.append("<" + TBL_COMMENT + ">\n");
-      createTabCommand.append("<" + LIST_PARTITIONS + ">\n");
-      createTabCommand.append("<" + SORT_BUCKET + ">\n");
-      createTabCommand.append("<" + SKEWED_INFO + ">\n");
-      createTabCommand.append("<" + ROW_FORMAT + ">\n");
-      if (needsLocation) {
-        createTabCommand.append("LOCATION\n");
-        createTabCommand.append("<" + TBL_LOCATION + ">\n");
-      }
-      createTabCommand.append("TBLPROPERTIES (\n");
-      createTabCommand.append("<" + TBL_PROPERTIES + ">)\n");
-      ST createTabStmt = new ST(createTabCommand.toString());
-
-      // For cases where the table is temporary
-      String tblTemp = "";
-      if (tbl.isTemporary()) {
-        duplicateProps.add("TEMPORARY");
-        tblTemp = "TEMPORARY ";
-      }
-      // For cases where the table is external
-      String tblExternal = "";
-      if (tbl.getTableType() == TableType.EXTERNAL_TABLE) {
-        duplicateProps.add("EXTERNAL");
-        tblExternal = "EXTERNAL ";
-      }
+  private String getCreateTableCommand(Table table) {
+    ST command = new ST(CREATE_TABLE_TEMPLATE);
 
-      // Columns
-      String tblColumns = "";
-      List<FieldSchema> cols = tbl.getCols();
-      List<String> columns = new ArrayList<String>();
-      for (FieldSchema col : cols) {
-        String columnDesc = "  `" + col.getName() + "` " + col.getType();
-        if (col.getComment() != null) {
-          columnDesc = columnDesc + " COMMENT '" + HiveStringUtils.escapeHiveCommand(col.getComment()) + "'";
-        }
-        columns.add(columnDesc);
-      }
-      tblColumns = StringUtils.join(columns, ", \n");
-
-      // Table comment
-      String tblComment = "";
-      String tabComment = tbl.getProperty("comment");
-      if (tabComment != null) {
-        duplicateProps.add("comment");
-        tblComment = "COMMENT '" + HiveStringUtils.escapeHiveCommand(tabComment) + "'";
-      }
+    command.add(NAME, desc.getTableName());
+    command.add(TEMPORARY, getTemporary(table));
+    command.add(EXTERNAL, getExternal(table));
+    command.add(LIST_COLUMNS, getColumns(table));
+    command.add(COMMENT, getComment(table));
+    command.add(PARTITIONS, getPartitions(table));
+    command.add(BUCKETS, getBuckets(table));
+    command.add(SKEWED, getSkewed(table));
+    command.add(ROW_FORMAT, getRowFormat(table));
+    command.add(LOCATION_BLOCK, getLocationBlock(table));
+    command.add(PROPERTIES, getProperties(table));
+
+    return command.render();
+  }
+
+  private String getTemporary(Table table) {
+    return table.isTemporary() ? "TEMPORARY " : "";
+  }
+
+  private String getExternal(Table table) {
+    return table.getTableType() == TableType.EXTERNAL_TABLE ? "EXTERNAL " : "";
+  }
 
-      // Partitions
-      String tblPartitions = "";
-      List<FieldSchema> partKeys = tbl.getPartitionKeys();
-      if (partKeys.size() > 0) {
-        tblPartitions += "PARTITIONED BY ( \n";
-        List<String> partCols = new ArrayList<String>();
-        for (FieldSchema partKey : partKeys) {
-          String partColDesc = "  `" + partKey.getName() + "` " + partKey.getType();
-          if (partKey.getComment() != null) {
-            partColDesc = partColDesc + " COMMENT '" + HiveStringUtils.escapeHiveCommand(partKey.getComment()) + "'";
-          }
-          partCols.add(partColDesc);
-        }
-        tblPartitions += StringUtils.join(partCols, ", \n");
-        tblPartitions += ")";
+  private String getColumns(Table table) {
+    List<String> columnDescs = new ArrayList<String>();
+    for (FieldSchema col : table.getCols()) {
+      String columnDesc = "  `" + col.getName() + "` " + col.getType();
+      if (col.getComment() != null) {
+        columnDesc += " COMMENT '" + HiveStringUtils.escapeHiveCommand(col.getComment()) + "'";
       }
+      columnDescs.add(columnDesc);
+    }
+    return StringUtils.join(columnDescs, ", \n");
+  }
+
+  private String getComment(Table table) {
+    String comment = table.getProperty("comment");
+    return (comment != null) ? "COMMENT '" + HiveStringUtils.escapeHiveCommand(comment) + "'" : "";
+  }
 
-      // Clusters (Buckets)
-      String tblSortBucket = "";
-      List<String> buckCols = tbl.getBucketCols();
-      if (buckCols.size() > 0) {
-        duplicateProps.add("SORTBUCKETCOLSPREFIX");
-        tblSortBucket += "CLUSTERED BY ( \n  ";
-        tblSortBucket += StringUtils.join(buckCols, ", \n  ");
-        tblSortBucket += ") \n";
-        List<Order> sortCols = tbl.getSortCols();
-        if (sortCols.size() > 0) {
-          tblSortBucket += "SORTED BY ( \n";
-          // Order
-          List<String> sortKeys = new ArrayList<String>();
-          for (Order sortCol : sortCols) {
-            String sortKeyDesc = "  " + sortCol.getCol() + " " + DirectionUtils.codeToText(sortCol.getOrder());
-            sortKeys.add(sortKeyDesc);
-          }
-          tblSortBucket += StringUtils.join(sortKeys, ", \n");
-          tblSortBucket += ") \n";
-        }
-        tblSortBucket += "INTO " + tbl.getNumBuckets() + " BUCKETS";
+  private String getPartitions(Table table) {
+    List<FieldSchema> partitionKeys = table.getPartitionKeys();
+    if (partitionKeys.isEmpty()) {
+      return "";
+    }
+
+    List<String> partitionDescs = new ArrayList<String>();
+    for (FieldSchema partitionKey : partitionKeys) {
+      String partitionDesc = "  `" + partitionKey.getName() + "` " + partitionKey.getType();
+      if (partitionKey.getComment() != null) {
+        partitionDesc += " COMMENT '" + HiveStringUtils.escapeHiveCommand(partitionKey.getComment()) + "'";
       }
+      partitionDescs.add(partitionDesc);
+    }
+    return "PARTITIONED BY ( \n" + StringUtils.join(partitionDescs, ", \n") + ")";
+  }
+
+  private String getBuckets(Table table) {
+    List<String> bucketCols = table.getBucketCols();
+    if (bucketCols.isEmpty()) {
+      return "";
+    }
+
+    String buckets = "CLUSTERED BY ( \n  " + StringUtils.join(bucketCols, ", \n  ") + ") \n";
 
-      // Skewed Info
-      StringBuilder tblSkewedInfo = new StringBuilder();
-      SkewedInfo skewedInfo = tbl.getSkewedInfo();
-      if (skewedInfo != null && !skewedInfo.getSkewedColNames().isEmpty()) {
-        tblSkewedInfo.append("SKEWED BY (" + StringUtils.join(skewedInfo.getSkewedColNames(), ",") + ")\n");
-        tblSkewedInfo.append("  ON (");
-        List<String> colValueList = new ArrayList<String>();
-        for (List<String> colValues : skewedInfo.getSkewedColValues()) {
-          colValueList.add("('" + StringUtils.join(colValues, "','") + "')");
-        }
-        tblSkewedInfo.append(StringUtils.join(colValueList, ",") + ")");
-        if (tbl.isStoredAsSubDirectories()) {
-          tblSkewedInfo.append("\n  STORED AS DIRECTORIES");
-        }
+    List<Order> sortColumns = table.getSortCols();
+    if (!sortColumns.isEmpty()) {
+      List<String> sortKeys = new ArrayList<String>();
+      for (Order sortColumn : sortColumns) {
+        String sortKeyDesc = "  " + sortColumn.getCol() + " " + DirectionUtils.codeToText(sortColumn.getOrder());
+        sortKeys.add(sortKeyDesc);
       }
+      buckets += "SORTED BY ( \n" + StringUtils.join(sortKeys, ", \n") + ") \n";
+    }
+
+    buckets += "INTO " + table.getNumBuckets() + " BUCKETS";
+    return buckets;
+  }
+
+  private String getSkewed(Table table) {
+    SkewedInfo skewedInfo = table.getSkewedInfo();
+    if (skewedInfo == null || skewedInfo.getSkewedColNames().isEmpty()) {
+      return "";
+    }
+
+    List<String> columnValuesList = new ArrayList<String>();
+    for (List<String> columnValues : skewedInfo.getSkewedColValues()) {
+      columnValuesList.add("('" + StringUtils.join(columnValues, "','") + "')");
+    }
 
-      // Row format (SerDe)
-      StringBuilder tblRowFormat = new StringBuilder();
-      StorageDescriptor sd = tbl.getTTable().getSd();
-      SerDeInfo serdeInfo = sd.getSerdeInfo();
-      Map<String, String> serdeParams = serdeInfo.getParameters();
-      tblRowFormat.append("ROW FORMAT SERDE \n");
-      tblRowFormat.append("  '" + HiveStringUtils.escapeHiveCommand(serdeInfo.getSerializationLib()) + "' \n");
-      if (tbl.getStorageHandler() == null) {
-        // If serialization.format property has the default value, it will not to be included in
-        // SERDE properties
-        if (Warehouse.DEFAULT_SERIALIZATION_FORMAT.equals(serdeParams.get(serdeConstants.SERIALIZATION_FORMAT))) {
-          serdeParams.remove(serdeConstants.SERIALIZATION_FORMAT);
-        }
-        if (!serdeParams.isEmpty()) {
-          appendSerdeParams(tblRowFormat, serdeParams).append(" \n");
-        }
-        tblRowFormat.append("STORED AS INPUTFORMAT \n  '"
-            + HiveStringUtils.escapeHiveCommand(sd.getInputFormat()) + "' \n");
-        tblRowFormat.append("OUTPUTFORMAT \n  '" + HiveStringUtils.escapeHiveCommand(sd.getOutputFormat()) + "'");
-      } else {
-        duplicateProps.add(META_TABLE_STORAGE);
-        tblRowFormat.append("STORED BY \n  '" +
-            HiveStringUtils.escapeHiveCommand(tbl.getParameters().get(META_TABLE_STORAGE)) + "' \n");
-        // SerDe Properties
-        if (!serdeParams.isEmpty()) {
-          appendSerdeParams(tblRowFormat, serdeInfo.getParameters());
-        }
+    String skewed =
+        "SKEWED BY (" + StringUtils.join(skewedInfo.getSkewedColNames(), ",") + ")\n" +
+        "  ON (" + StringUtils.join(columnValuesList, ",") + ")";
+    if (table.isStoredAsSubDirectories()) {
+      skewed += "\n  STORED AS DIRECTORIES";
+    }
+    return skewed;
+  }
+
+  private String getRowFormat(Table table) {
+    StringBuilder rowFormat = new StringBuilder();
+
+    StorageDescriptor sd = table.getTTable().getSd();
+    SerDeInfo serdeInfo = sd.getSerdeInfo();
+
+    rowFormat
+      .append("ROW FORMAT SERDE \n")
+      .append("  '" + HiveStringUtils.escapeHiveCommand(serdeInfo.getSerializationLib()) + "' \n");
+
+    Map<String, String> serdeParams = serdeInfo.getParameters();
+    if (table.getStorageHandler() == null) {
+      // If serialization.format property has the default value, it will not to be included in SERDE properties
+      if (Warehouse.DEFAULT_SERIALIZATION_FORMAT.equals(serdeParams.get(serdeConstants.SERIALIZATION_FORMAT))) {
+        serdeParams.remove(serdeConstants.SERIALIZATION_FORMAT);
       }
-      String tblLocation = "  '" + HiveStringUtils.escapeHiveCommand(sd.getLocation()) + "'";
-
-      // Table properties
-      duplicateProps.addAll(StatsSetupConst.TABLE_PARAMS_STATS_KEYS);
-      String tblProperties = DDLUtils.propertiesToString(tbl.getParameters(), duplicateProps);
-
-      createTabStmt.add(TEMPORARY, tblTemp);
-      createTabStmt.add(EXTERNAL, tblExternal);
-      createTabStmt.add(LIST_COLUMNS, tblColumns);
-      createTabStmt.add(TBL_COMMENT, tblComment);
-      createTabStmt.add(LIST_PARTITIONS, tblPartitions);
-      createTabStmt.add(SORT_BUCKET, tblSortBucket);
-      createTabStmt.add(SKEWED_INFO, tblSkewedInfo);
-      createTabStmt.add(ROW_FORMAT, tblRowFormat);
-      // Table location should not be printed with hbase backed tables
-      if (needsLocation) {
-        createTabStmt.add(TBL_LOCATION, tblLocation);
+      if (!serdeParams.isEmpty()) {
+        appendSerdeParams(rowFormat, serdeParams);
+        rowFormat.append(" \n");
+      }
+      rowFormat
+        .append("STORED AS INPUTFORMAT \n  '" + HiveStringUtils.escapeHiveCommand(sd.getInputFormat()) + "' \n")
+        .append("OUTPUTFORMAT \n  '" + HiveStringUtils.escapeHiveCommand(sd.getOutputFormat()) + "'");
+    } else {
+      String metaTableStorage = table.getParameters().get(META_TABLE_STORAGE);
+      rowFormat.append("STORED BY \n  '" + HiveStringUtils.escapeHiveCommand(metaTableStorage) + "' \n");
+      if (!serdeParams.isEmpty()) {
+        appendSerdeParams(rowFormat, serdeInfo.getParameters());
       }
-      createTabStmt.add(TBL_PROPERTIES, tblProperties);
-
-      outStream.write(createTabStmt.render().getBytes(StandardCharsets.UTF_8));
-    } catch (IOException e) {
-      LOG.info("show create table: ", e);
-      return 1;
     }
 
-    return 0;
+    return rowFormat.toString();
   }
 
-  public static StringBuilder appendSerdeParams(StringBuilder builder, Map<String, String> serdeParam) {
-    serdeParam = new TreeMap<String, String>(serdeParam);
-    builder.append("WITH SERDEPROPERTIES ( \n");
+  public static void appendSerdeParams(StringBuilder builder, Map<String, String> serdeParams) {
+    SortedMap<String, String> sortedSerdeParams = new TreeMap<String, String>(serdeParams);
     List<String> serdeCols = new ArrayList<String>();
-    for (Entry<String, String> entry : serdeParam.entrySet()) {
+    for (Entry<String, String> entry : sortedSerdeParams.entrySet()) {
       serdeCols.add("  '" + entry.getKey() + "'='" + HiveStringUtils.escapeHiveCommand(entry.getValue()) + "'");
     }
-    builder.append(StringUtils.join(serdeCols, ", \n")).append(')');
-    return builder;
+
+    builder
+      .append("WITH SERDEPROPERTIES ( \n")
+      .append(StringUtils.join(serdeCols, ", \n"))
+      .append(')');
+  }
+
+  private static final String CREATE_TABLE_TEMPLATE_LOCATION =
+      "LOCATION\n" +
+      "<" + LOCATION + ">\n";
+
+  private String getLocationBlock(Table table) {
+    if (!CreateTableOperation.doesTableNeedLocation(table)) {
+      return "";
+    }
+
+    ST locationBlock = new ST(CREATE_TABLE_TEMPLATE_LOCATION);
+    StorageDescriptor sd = table.getTTable().getSd();
+    locationBlock.add(LOCATION, "  '" + HiveStringUtils.escapeHiveCommand(sd.getLocation()) + "'");
+    return locationBlock.render();
+  }
+
+  private static final Set<String> PROPERTIES_TO_IGNORE_AT_TBLPROPERTIES = Sets.union(
+      ImmutableSet.<String>of("TEMPORARY", "EXTERNAL", "comment", "SORTBUCKETCOLSPREFIX", META_TABLE_STORAGE),
+      new HashSet<String>(StatsSetupConst.TABLE_PARAMS_STATS_KEYS));
+
+  private String getProperties(Table table) {
+    return DDLUtils.propertiesToString(table.getParameters(), PROPERTIES_TO_IGNORE_AT_TBLPROPERTIES);
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java
index d48ae04..04d0aa1 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/info/DescTableOperation.java
@@ -63,7 +63,7 @@ import org.apache.hadoop.hive.serde2.Deserializer;
 import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo;
 import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoUtils;
 
-import avro.shaded.com.google.common.collect.Lists;
+import com.google.common.collect.Lists;
 
 /**
  * Operation process of dropping a table.
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
index 2636530..69414f2 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/lock/ShowLocksOperation.java
@@ -185,7 +185,7 @@ public class ShowLocksOperation extends DDLOperation<ShowLocksDesc> {
   public static void dumpLockInfo(DataOutputStream os, ShowLocksResponse response) throws IOException {
     SessionState sessionState = SessionState.get();
     // Write a header for CliDriver
-    if(!sessionState.isHiveServerQuery()) {
+    if (!sessionState.isHiveServerQuery()) {
       os.writeBytes("Lock ID");
       os.write(Utilities.tabCode);
       os.writeBytes("Database");
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/AlterTableSetPropertiesOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/AlterTableSetPropertiesOperation.java
index d25bf3c..72db457 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/AlterTableSetPropertiesOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/misc/AlterTableSetPropertiesOperation.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.hive.common.StatsSetupConst;
 import org.apache.hadoop.hive.conf.HiveConf;
 import org.apache.hadoop.hive.conf.HiveConf.ConfVars;
 import org.apache.hadoop.hive.metastore.TableType;
+import org.apache.hadoop.hive.metastore.conf.MetastoreConf;
 import org.apache.hadoop.hive.ql.ddl.DDLOperationContext;
 import org.apache.hadoop.hive.ql.ddl.table.AbstractAlterTableOperation;
 import org.apache.hadoop.hive.ql.exec.Task;
@@ -70,7 +71,7 @@ public class AlterTableSetPropertiesOperation extends AbstractAlterTableOperatio
         } else {
           if (!table.getPartitionKeys().isEmpty()) {
             PartitionIterable parts = new PartitionIterable(context.getDb(), table, null,
-                HiveConf.getIntVar(context.getConf(), ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
+                MetastoreConf.getIntVar(context.getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX));
             for (Partition part : parts) {
               checkMmLb(part);
             }
@@ -112,7 +113,7 @@ public class AlterTableSetPropertiesOperation extends AbstractAlterTableOperatio
 
     if (!table.getPartitionKeys().isEmpty()) {
       PartitionIterable parts = new PartitionIterable(context.getDb(), table, null,
-          HiveConf.getIntVar(context.getConf(), ConfVars.METASTORE_BATCH_RETRIEVE_MAX));
+          MetastoreConf.getIntVar(context.getConf(), MetastoreConf.ConfVars.BATCH_RETRIEVE_MAX));
       for (Partition part : parts) {
         checkMmLb(part);
         Path source = part.getDataLocation();
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableAddPartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableAddPartitionDesc.java
index 72828ef..e1aecbc 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableAddPartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableAddPartitionDesc.java
@@ -329,8 +329,8 @@ public class AlterTableAddPartitionDesc implements DDLDesc, Serializable {
    * @return what kind of replication scope this drop is running under.
    * This can result in a "CREATE/REPLACE IF NEWER THAN" kind of semantic
    */
-  public ReplicationSpec getReplicationSpec(){
-    if (replicationSpec == null){
+  public ReplicationSpec getReplicationSpec() {
+    if (replicationSpec == null) {
       this.replicationSpec = new ReplicationSpec();
     }
     return this.replicationSpec;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionDesc.java
index 01e505a..9903264 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionDesc.java
@@ -96,7 +96,7 @@ public class AlterTableDropPartitionDesc implements DDLDesc, Serializable {
    * @return what kind of replication scope this drop is running under.
    * This can result in a "DROP IF OLDER THAN" kind of semantic
    */
-  public ReplicationSpec getReplicationSpec(){
+  public ReplicationSpec getReplicationSpec() {
     return replicationSpec;
   }
 }
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
index c084662..e7cc6d3 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/partition/AlterTableDropPartitionOperation.java
@@ -84,14 +84,14 @@ public class AlterTableDropPartitionOperation extends DDLOperation<AlterTableDro
       return;
     }
 
-    for (AlterTableDropPartitionDesc.PartitionDesc partSpec : desc.getPartSpecs()){
+    for (AlterTableDropPartitionDesc.PartitionDesc partSpec : desc.getPartSpecs()) {
       List<Partition> partitions = new ArrayList<>();
       try {
         context.getDb().getPartitionsByExpr(tbl, partSpec.getPartSpec(), context.getConf(), partitions);
         for (Partition p : Iterables.filter(partitions, replicationSpec.allowEventReplacementInto())) {
           context.getDb().dropPartition(tbl.getDbName(), tbl.getTableName(), p.getValues(), true);
         }
-      } catch (NoSuchObjectException e){
+      } catch (NoSuchObjectException e) {
         // ignore NSOE because that means there's nothing to drop.
       } catch (Exception e) {
         throw new HiveException(e.getMessage(), e);
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java
index 69f5f23..315857b 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableArchiveOperation.java
@@ -130,7 +130,7 @@ public class AlterTableArchiveOperation extends DDLOperation<AlterTableArchiveDe
       throw new HiveException("No partition matches the specification");
     } else if (partitionSpecInfo.values.size() != table.getPartCols().size()) {
       // for partial specifications we need partitions to follow the scheme
-      for (Partition partition : partitions){
+      for (Partition partition : partitions) {
         if (AlterTableArchiveUtils.partitionInCustomLocation(table, partition)) {
           throw new HiveException(String.format("ARCHIVE cannot run for partition groups with custom locations like %s",
               partition.getLocation()));
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java
index 3039e29..0fbaea0 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/table/storage/AlterTableUnarchiveOperation.java
@@ -147,7 +147,7 @@ public class AlterTableUnarchiveOperation extends DDLOperation<AlterTableUnarchi
       throw new HiveException("No partition matches the specification");
     } else if (partitionSpecInfo.values.size() != table.getPartCols().size()) {
       // for partial specifications we need partitions to follow the scheme
-      for (Partition partition : partitions){
+      for (Partition partition : partitions) {
         if (AlterTableArchiveUtils.partitionInCustomLocation(table, partition)) {
           String message = String.format("UNARCHIVE cannot run for partition groups with custom locations like %s",
               partition.getLocation());
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewDesc.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewDesc.java
index e51465e..31bb7ca 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewDesc.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewDesc.java
@@ -350,8 +350,8 @@ public class CreateViewDesc implements DDLDesc, Serializable {
   /**
    * @return what kind of replication spec this create is running under.
    */
-  public ReplicationSpec getReplicationSpec(){
-    if (replicationSpec == null){
+  public ReplicationSpec getReplicationSpec() {
+    if (replicationSpec == null) {
       this.replicationSpec = new ReplicationSpec();
     }
     return this.replicationSpec;
diff --git a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewOperation.java b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewOperation.java
index 3f6f999..2176fd5 100644
--- a/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewOperation.java
+++ b/ql/src/java/org/apache/hadoop/hive/ql/ddl/view/CreateViewOperation.java
@@ -48,7 +48,7 @@ public class CreateViewOperation extends DDLOperation<CreateViewDesc> {
       // Check whether we are replicating
       if (desc.getReplicationSpec().isInReplicationScope()) {
         // if this is a replication spec, then replace-mode semantics might apply.
-        if (desc.getReplicationSpec().allowEventReplacementInto(oldview.getParameters())){
+        if (desc.getReplicationSpec().allowEventReplacementInto(oldview.getParameters())) {
           desc.setReplace(true); // we replace existing view.
         } else {
           LOG.debug("DDLTask: Create View is skipped as view {} is newer than update",
diff --git a/ql/src/test/results/clientpositive/show_create_table_alter.q.out b/ql/src/test/results/clientpositive/show_create_table_alter.q.out
index 2c75c36..b9b43e8 100644
--- a/ql/src/test/results/clientpositive/show_create_table_alter.q.out
+++ b/ql/src/test/results/clientpositive/show_create_table_alter.q.out
@@ -65,7 +65,6 @@ OUTPUTFORMAT
 LOCATION
 #### A masked pattern was here ####
 TBLPROPERTIES (
-  'EXTERNAL'='FALSE', 
   'bucketing_version'='2', 
 #### A masked pattern was here ####
 PREHOOK: query: ALTER TABLE tmp_showcrt1_n1 SET TBLPROPERTIES ('comment'='changed comment', 'EXTERNAL'='TRUE')