You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by rh...@apache.org on 2014/03/25 23:14:52 UTC

svn commit: r1581559 - in /hive/branches/branch-0.13/ql/src: java/org/apache/hadoop/hive/ql/exec/ java/org/apache/hadoop/hive/ql/metadata/ java/org/apache/hadoop/hive/ql/parse/ java/org/apache/hadoop/hive/ql/plan/ java/org/apache/hadoop/hive/ql/securit...

Author: rhbutani
Date: Tue Mar 25 22:14:52 2014
New Revision: 1581559

URL: http://svn.apache.org/r1581559
Log:
HIVE-6460 Need new 'show' functionality for transactions (Alan Gates via Harish Butani)

Added:
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCompactionsDesc.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTxnsDesc.java
    hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q
    hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact1.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact2.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact3.q.out
    hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_showlocks.q.out
Modified:
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowLocksDesc.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
    hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/exec/DDLTask.java Tue Mar 25 22:14:52 2014
@@ -62,6 +62,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -75,10 +76,13 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.PrivilegeGrantInfo;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponse;
 import org.apache.hadoop.hive.metastore.api.ShowLocksResponseElement;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.StorageDescriptor;
+import org.apache.hadoop.hive.metastore.api.TxnInfo;
 import org.apache.hadoop.hive.ql.Context;
 import org.apache.hadoop.hive.ql.DriverContext;
 import org.apache.hadoop.hive.ql.ErrorMsg;
@@ -142,6 +146,7 @@ import org.apache.hadoop.hive.ql.plan.Re
 import org.apache.hadoop.hive.ql.plan.RevokeDesc;
 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.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -152,6 +157,7 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTblPropertiesDesc;
+import org.apache.hadoop.hive.ql.plan.ShowTxnsDesc;
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.TruncateTableDesc;
 import org.apache.hadoop.hive.ql.plan.UnlockDatabaseDesc;
@@ -330,6 +336,8 @@ public class DDLTask extends Task<DDLWor
           return archive(db, simpleDesc, driverContext);
         } else if (simpleDesc.getType() == AlterTableTypes.UNARCHIVE) {
           return unarchive(db, simpleDesc);
+        } else if (simpleDesc.getType() == AlterTableTypes.COMPACT) {
+          return compact(db, simpleDesc);
         }
       }
 
@@ -383,7 +391,17 @@ public class DDLTask extends Task<DDLWor
         return showLocks(showLocks);
       }
 
-      LockTableDesc lockTbl = work.getLockTblDesc();
+      ShowCompactionsDesc compactionsDesc = work.getShowCompactionsDesc();
+      if (compactionsDesc != null) {
+        return showCompactions(compactionsDesc);
+      }
+
+      ShowTxnsDesc txnsDesc = work.getShowTxnsDesc();
+      if (txnsDesc != null) {
+        return showTxns(txnsDesc);
+      }
+
+       LockTableDesc lockTbl = work.getLockTblDesc();
       if (lockTbl != null) {
         return lockTable(lockTbl);
       }
@@ -1895,6 +1913,34 @@ public class DDLTask extends Task<DDLWor
     }
   }
 
+  private int compact(Hive db, AlterTableSimpleDesc desc) throws HiveException {
+
+    String dbName = desc.getDbName();
+    String tblName = desc.getTableName();
+
+    Table tbl = db.getTable(dbName, tblName);
+
+    String partName = null;
+    if (desc.getPartSpec() == null) {
+      // Compaction can only be done on the whole table if the table is non-partitioned.
+      if (tbl.isPartitioned()) {
+        throw new HiveException(ErrorMsg.NO_COMPACTION_PARTITION);
+      }
+    } else {
+      Map<String, String> partSpec = desc.getPartSpec();
+      List<Partition> partitions = db.getPartitions(tbl, partSpec);
+      if (partitions.size() > 1) {
+        throw new HiveException(ErrorMsg.TOO_MANY_COMPACTION_PARTITIONS);
+      } else if (partitions.size() == 0) {
+        throw new HiveException(ErrorMsg.INVALID_PARTITION_SPEC);
+      }
+      partName = partitions.get(0).getName();
+    }
+    db.compact(tbl.getDbName(), tbl.getTableName(), partName, desc.getCompactionType());
+    console.printInfo("Compaction enqueued.");
+    return 0;
+  }
+
   /**
    * MetastoreCheck, see if the data in the metastore matches what is on the
    * dfs. Current version checks for tables and partitions that are either
@@ -2567,7 +2613,6 @@ public class DDLTask extends Task<DDLWor
         locks = lockMgr.getLocks(false, isExt);
       }
       else {
-        // TODO make this work
         locks = lockMgr.getLocks(getHiveObject(showLocks.getTableName(),
             showLocks.getPartSpec()),
             true, isExt);
@@ -2719,6 +2764,102 @@ public class DDLTask extends Task<DDLWor
     return 0;
   }
 
+  private int showCompactions(ShowCompactionsDesc desc) throws HiveException {
+    // Call the metastore to get the currently queued and running compactions.
+    ShowCompactResponse rsp = db.showCompactions();
+
+    // Write the results into the file
+    DataOutputStream os = null;
+    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);
+      os.writeBytes("Table");
+      os.write(separator);
+      os.writeBytes("Partition");
+      os.write(separator);
+      os.writeBytes("Type");
+      os.write(separator);
+      os.writeBytes("State");
+      os.write(separator);
+      os.writeBytes("Worker");
+      os.write(separator);
+      os.writeBytes("Start Time");
+      os.write(terminator);
+
+      for (ShowCompactResponseElement e : rsp.getCompacts()) {
+        os.writeBytes(e.getDbname());
+        os.write(separator);
+        os.writeBytes(e.getTablename());
+        os.write(separator);
+        String part = e.getPartitionname();
+        os.writeBytes(part == null ? "NULL" : part);
+        os.write(separator);
+        os.writeBytes(e.getType().toString());
+        os.write(separator);
+        os.writeBytes(e.getState());
+        os.write(separator);
+        String wid = e.getWorkerid();
+        os.writeBytes(wid == null ? "NULL" : wid);
+        os.write(separator);
+        os.writeBytes(Long.toString(e.getStart()));
+        os.write(terminator);
+      }
+      os.close();
+    } catch (IOException e) {
+      LOG.warn("show compactions: " + stringifyException(e));
+      return 1;
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream)os);
+    }
+    return 0;
+  }
+
+  private int showTxns(ShowTxnsDesc desc) throws HiveException {
+    // Call the metastore to get the currently queued and running compactions.
+    GetOpenTxnsInfoResponse rsp = db.showTransactions();
+
+    // Write the results into the file
+    DataOutputStream os = null;
+    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);
+      os.writeBytes("Transaction State");
+      os.write(separator);
+      os.writeBytes("User");
+      os.write(separator);
+      os.writeBytes("Hostname");
+      os.write(terminator);
+
+      for (TxnInfo txn : rsp.getOpen_txns()) {
+        os.writeBytes(Long.toString(txn.getId()));
+        os.write(separator);
+        os.writeBytes(txn.getState().toString());
+        os.write(separator);
+        os.writeBytes(txn.getUser());
+        os.write(separator);
+        os.writeBytes(txn.getHostname());
+        os.write(terminator);
+      }
+      os.close();
+    } catch (IOException e) {
+      LOG.warn("show transactions: " + stringifyException(e));
+      return 1;
+    } finally {
+      IOUtils.closeStream((FSDataOutputStream)os);
+    }
+    return 0;
+  }
+
    /**
    * Lock the table/partition specified
    *

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java Tue Mar 25 22:14:52 2014
@@ -67,9 +67,11 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.AlreadyExistsException;
 import org.apache.hadoop.hive.metastore.api.ColumnStatistics;
 import org.apache.hadoop.hive.metastore.api.ColumnStatisticsObj;
+import org.apache.hadoop.hive.metastore.api.CompactionType;
 import org.apache.hadoop.hive.metastore.api.Database;
 import org.apache.hadoop.hive.metastore.api.FieldSchema;
 import org.apache.hadoop.hive.metastore.api.Function;
+import org.apache.hadoop.hive.metastore.api.GetOpenTxnsInfoResponse;
 import org.apache.hadoop.hive.metastore.api.HiveObjectPrivilege;
 import org.apache.hadoop.hive.metastore.api.HiveObjectRef;
 import org.apache.hadoop.hive.metastore.api.HiveObjectType;
@@ -83,6 +85,7 @@ import org.apache.hadoop.hive.metastore.
 import org.apache.hadoop.hive.metastore.api.PrivilegeBag;
 import org.apache.hadoop.hive.metastore.api.Role;
 import org.apache.hadoop.hive.metastore.api.SerDeInfo;
+import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
 import org.apache.hadoop.hive.metastore.api.SkewedInfo;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.ql.exec.Utilities;
@@ -2568,6 +2571,47 @@ private void constructOneLBLocationMap(F
     }
   }
 
+  /**
+   * Enqueue a compaction request.
+   * @param dbname name of the database, if null default will be used.
+   * @param tableName name of the table, cannot be null
+   * @param partName name of the partition, if null table will be compacted (valid only for
+   *                 non-partitioned tables).
+   * @param compactType major or minor
+   * @throws HiveException
+   */
+  public void compact(String dbname, String tableName, String partName,  String compactType)
+      throws HiveException {
+    try {
+      CompactionType cr = null;
+      if ("major".equals(compactType)) cr = CompactionType.MAJOR;
+      else if ("minor".equals(compactType)) cr = CompactionType.MINOR;
+      else throw new RuntimeException("Unknown compaction type " + compactType);
+      getMSC().compact(dbname, tableName, partName, cr);
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
+  public ShowCompactResponse showCompactions() throws HiveException {
+    try {
+      return getMSC().showCompactions();
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
+  public GetOpenTxnsInfoResponse showTransactions() throws HiveException {
+    try {
+      return getMSC().showTxns();
+    } catch (Exception e) {
+      LOG.error(StringUtils.stringifyException(e));
+      throw new HiveException(e);
+    }
+  }
+
   public static String[] getQualifiedNames(String qualifiedName) {
     return qualifiedName.split("\\.");
   }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/DDLSemanticAnalyzer.java Tue Mar 25 22:14:52 2014
@@ -70,6 +70,9 @@ import org.apache.hadoop.hive.ql.index.H
 import org.apache.hadoop.hive.ql.io.IgnoreKeyTextOutputFormat;
 import org.apache.hadoop.hive.ql.io.RCFileInputFormat;
 import org.apache.hadoop.hive.ql.lib.Node;
+import org.apache.hadoop.hive.ql.lockmgr.HiveTxnManager;
+import org.apache.hadoop.hive.ql.lockmgr.LockException;
+import org.apache.hadoop.hive.ql.lockmgr.TxnManagerFactory;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
 import org.apache.hadoop.hive.ql.metadata.HiveUtils;
@@ -112,6 +115,7 @@ import org.apache.hadoop.hive.ql.plan.Pr
 import org.apache.hadoop.hive.ql.plan.RenamePartitionDesc;
 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.ShowCreateTableDesc;
 import org.apache.hadoop.hive.ql.plan.ShowDatabasesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowFunctionsDesc;
@@ -122,6 +126,7 @@ import org.apache.hadoop.hive.ql.plan.Sh
 import org.apache.hadoop.hive.ql.plan.ShowTableStatusDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTablesDesc;
 import org.apache.hadoop.hive.ql.plan.ShowTblPropertiesDesc;
+import org.apache.hadoop.hive.ql.plan.ShowTxnsDesc;
 import org.apache.hadoop.hive.ql.plan.StatsWork;
 import org.apache.hadoop.hive.ql.plan.SwitchDatabaseDesc;
 import org.apache.hadoop.hive.ql.plan.TableDesc;
@@ -262,6 +267,8 @@ public class DDLSemanticAnalyzer extends
         analyzeAlterTableBucketNum(ast, tableName, partSpec);
       } else if (ast.getToken().getType() == HiveParser.TOK_ALTERTABLE_CLUSTER_SORT) {
         analyzeAlterTableClusterSort(ast, tableName, partSpec);
+      } else if (ast.getToken().getType() == HiveParser.TOK_COMPACT) {
+        analyzeAlterTableCompact(ast, tableName, partSpec);
       }
       break;
     }
@@ -313,6 +320,14 @@ public class DDLSemanticAnalyzer extends
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeShowDbLocks(ast);
       break;
+    case HiveParser.TOK_SHOW_COMPACTIONS:
+      ctx.setResFile(ctx.getLocalTmpPath());
+      analyzeShowCompactions(ast);
+      break;
+    case HiveParser.TOK_SHOW_TRANSACTIONS:
+      ctx.setResFile(ctx.getLocalTmpPath());
+      analyzeShowTxns(ast);
+      break;
     case HiveParser.TOK_DESCFUNCTION:
       ctx.setResFile(ctx.getLocalTmpPath());
       analyzeDescFunction(ast);
@@ -1626,6 +1641,24 @@ public class DDLSemanticAnalyzer extends
     }
   }
 
+  private void analyzeAlterTableCompact(ASTNode ast, String tableName,
+      HashMap<String, String> partSpec) throws SemanticException {
+
+    String type = unescapeSQLString(ast.getChild(0).getText()).toLowerCase();
+
+    if (!type.equals("minor") && !type.equals("major")) {
+      throw new SemanticException(ErrorMsg.INVALID_COMPACTION_TYPE.getMsg());
+    }
+
+    LinkedHashMap<String, String> newPartSpec = null;
+    if (partSpec != null) newPartSpec = new LinkedHashMap<String, String>(partSpec);
+
+    AlterTableSimpleDesc desc = new AlterTableSimpleDesc(SessionState.get().getCurrentDatabase(),
+        tableName, newPartSpec, type);
+
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+  }
+
   static HashMap<String, String> getProps(ASTNode prop) {
     HashMap<String, String> mapProp = new HashMap<String, String>();
     readProps(prop, mapProp);
@@ -2217,8 +2250,15 @@ public class DDLSemanticAnalyzer extends
       }
     }
 
+    HiveTxnManager txnManager = null;
+    try {
+      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    } catch (LockException e) {
+      throw new SemanticException(e.getMessage());
+    }
+
     ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), tableName,
-        partSpec, isExtended);
+        partSpec, isExtended, txnManager.useNewShowLocksFormat());
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         showLocksDesc), conf));
     setFetchTask(createFetchTask(showLocksDesc.getSchema()));
@@ -2240,8 +2280,15 @@ public class DDLSemanticAnalyzer extends
     boolean isExtended = (ast.getChildCount() > 1);
     String dbName = stripQuotes(ast.getChild(0).getText());
 
+    HiveTxnManager txnManager = null;
+    try {
+      txnManager = TxnManagerFactory.getTxnManagerFactory().getTxnManager(conf);
+    } catch (LockException e) {
+      throw new SemanticException(e.getMessage());
+    }
+
     ShowLocksDesc showLocksDesc = new ShowLocksDesc(ctx.getResFile(), dbName,
-                                                    isExtended);
+                                                    isExtended, txnManager.useNewShowLocksFormat());
     rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(),
         showLocksDesc), conf));
     setFetchTask(createFetchTask(showLocksDesc.getSchema()));
@@ -2283,6 +2330,28 @@ public class DDLSemanticAnalyzer extends
   }
 
   /**
+   * Add a task to execute "SHOW COMPACTIONS"
+   * @param ast The parsed command tree.
+   * @throws SemanticException Parsing failed.
+   */
+  private void analyzeShowCompactions(ASTNode ast) throws SemanticException {
+    ShowCompactionsDesc desc = new ShowCompactionsDesc(ctx.getResFile());
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    setFetchTask(createFetchTask(desc.getSchema()));
+  }
+
+  /**
+   * Add a task to execute "SHOW COMPACTIONS"
+   * @param ast The parsed command tree.
+   * @throws SemanticException Parsing failed.
+   */
+  private void analyzeShowTxns(ASTNode ast) throws SemanticException {
+    ShowTxnsDesc desc = new ShowTxnsDesc(ctx.getResFile());
+    rootTasks.add(TaskFactory.get(new DDLWork(getInputs(), getOutputs(), desc), conf));
+    setFetchTask(createFetchTask(desc.getSchema()));
+  }
+
+   /**
    * Add the task according to the parsed command tree. This is used for the CLI
    * command "UNLOCK TABLE ..;".
    *

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveLexer.g Tue Mar 25 22:14:52 2014
@@ -291,7 +291,9 @@ KW_EXCHANGE: 'EXCHANGE';
 KW_ADMIN: 'ADMIN';
 KW_OWNER: 'OWNER';
 KW_PRINCIPALS: 'PRINCIPALS';
-
+KW_COMPACT: 'COMPACT';
+KW_COMPACTIONS: 'COMPACTIONS';
+KW_TRANSACTIONS: 'TRANSACTIONS';
 
 // Operators
 // NOTE: if you add a new function/operator, add it to sysFuncNames so that describe function _FUNC_ will work.

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/HiveParser.g Tue Mar 25 22:14:52 2014
@@ -328,6 +328,9 @@ TOK_FILE;
 TOK_JAR;
 TOK_RESOURCE_URI;
 TOK_RESOURCE_LIST;
+TOK_COMPACT;
+TOK_SHOW_COMPACTIONS;
+TOK_SHOW_TRANSACTIONS;
 }
 
 
@@ -1123,6 +1126,7 @@ alterTblPartitionStatementSuffix
   | alterStatementSuffixBucketNum
   | alterTblPartitionStatementSuffixSkewedLocation
   | alterStatementSuffixClusterbySortby
+  | alterStatementSuffixCompact
   ;
 
 alterStatementSuffixFileFormat
@@ -1239,6 +1243,14 @@ alterStatementSuffixBucketNum
     -> ^(TOK_TABLEBUCKETS $num)
     ;
 
+alterStatementSuffixCompact
+@init { msgs.push("compaction request"); }
+@after { msgs.pop(); }
+    : KW_COMPACT compactType=StringLiteral
+    -> ^(TOK_COMPACT $compactType)
+    ;
+
+
 fileFormat
 @init { pushMsg("file format specification", state); }
 @after { popMsg(state); }
@@ -1309,6 +1321,8 @@ showStatement
     | KW_SHOW KW_LOCKS KW_DATABASE (dbName=Identifier) (isExtended=KW_EXTENDED)? -> ^(TOK_SHOWDBLOCKS $dbName $isExtended?)
     | KW_SHOW (showOptions=KW_FORMATTED)? (KW_INDEX|KW_INDEXES) KW_ON showStmtIdentifier ((KW_FROM|KW_IN) db_name=identifier)?
     -> ^(TOK_SHOWINDEXES showStmtIdentifier $showOptions? $db_name?)
+    | KW_SHOW KW_COMPACTIONS -> ^(TOK_SHOW_COMPACTIONS)
+    | KW_SHOW KW_TRANSACTIONS -> ^(TOK_SHOW_TRANSACTIONS)
     ;
 
 lockStatement

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/IdentifiersParser.g Tue Mar 25 22:14:52 2014
@@ -544,5 +544,5 @@ functionIdentifier
 
 nonReserved
     :
-    KW_TRUE | KW_FALSE | KW_LIKE | KW_EXISTS | KW_ASC | KW_DESC | KW_ORDER | KW_GROUP | KW_BY | KW_AS | KW_INSERT | KW_OVERWRITE | KW_OUTER | KW_LEFT | KW_RIGHT | KW_FULL | KW_PARTITION | KW_PARTITIONS | KW_TABLE | KW_TABLES | KW_COLUMNS | KW_INDEX | KW_INDEXES | KW_REBUILD | KW_FUNCTIONS | KW_SHOW | KW_MSCK | KW_REPAIR | KW_DIRECTORY | KW_LOCAL | KW_USING | KW_CLUSTER | KW_DISTRIBUTE | KW_SORT | KW_UNION | KW_LOAD | KW_EXPORT | KW_IMPORT | KW_DATA | KW_INPATH | KW_IS | KW_NULL | KW_CREATE | KW_EXTERNAL | KW_ALTER | KW_CHANGE | KW_FIRST | KW_AFTER | KW_DESCRIBE | KW_DROP | KW_RENAME | KW_IGNORE | KW_PROTECTION | KW_TO | KW_COMMENT | KW_BOOLEAN | KW_TINYINT | KW_SMALLINT | KW_INT | KW_BIGINT | KW_FLOAT | KW_DOUBLE | KW_DATE | KW_DATETIME | KW_TIMESTAMP | KW_DECIMAL | KW_STRING | KW_ARRAY | KW_STRUCT | KW_UNIONTYPE | KW_PARTITIONED | KW_CLUSTERED | KW_SORTED | KW_INTO | KW_BUCKETS | KW_ROW | KW_ROWS | KW_FORMAT | KW_DELIMITED | KW_FIELDS | KW_TERMINATED | KW_ESCAPED | KW_COLLECTION | 
 KW_ITEMS | KW_KEYS | KW_KEY_TYPE | KW_LINES | KW_STORED | KW_FILEFORMAT | KW_SEQUENCEFILE | KW_TEXTFILE | KW_RCFILE | KW_ORCFILE | KW_PARQUETFILE | KW_INPUTFORMAT | KW_OUTPUTFORMAT | KW_INPUTDRIVER | KW_OUTPUTDRIVER | KW_OFFLINE | KW_ENABLE | KW_DISABLE | KW_READONLY | KW_NO_DROP | KW_LOCATION | KW_BUCKET | KW_OUT | KW_OF | KW_PERCENT | KW_ADD | KW_REPLACE | KW_RLIKE | KW_REGEXP | KW_TEMPORARY | KW_EXPLAIN | KW_FORMATTED | KW_PRETTY | KW_DEPENDENCY | KW_LOGICAL | KW_SERDE | KW_WITH | KW_DEFERRED | KW_SERDEPROPERTIES | KW_DBPROPERTIES | KW_LIMIT | KW_SET | KW_UNSET | KW_TBLPROPERTIES | KW_IDXPROPERTIES | KW_VALUE_TYPE | KW_ELEM_TYPE | KW_MAPJOIN | KW_STREAMTABLE | KW_HOLD_DDLTIME | KW_CLUSTERSTATUS | KW_UTC | KW_UTCTIMESTAMP | KW_LONG | KW_DELETE | KW_PLUS | KW_MINUS | KW_FETCH | KW_INTERSECT | KW_VIEW | KW_IN | KW_DATABASES | KW_MATERIALIZED | KW_SCHEMA | KW_SCHEMAS | KW_GRANT | KW_REVOKE | KW_SSL | KW_UNDO | KW_LOCK | KW_LOCKS | KW_UNLOCK | KW_SHARED | KW_EXCLUSIVE | KW_PROCEDURE |
  KW_UNSIGNED | KW_WHILE | KW_READ | KW_READS | KW_PURGE | KW_RANGE | KW_ANALYZE | KW_BEFORE | KW_BETWEEN | KW_BOTH | KW_BINARY | KW_CONTINUE | KW_CURSOR | KW_TRIGGER | KW_RECORDREADER | KW_RECORDWRITER | KW_SEMI | KW_LATERAL | KW_TOUCH | KW_ARCHIVE | KW_UNARCHIVE | KW_COMPUTE | KW_STATISTICS | KW_USE | KW_OPTION | KW_CONCATENATE | KW_SHOW_DATABASE | KW_UPDATE | KW_RESTRICT | KW_CASCADE | KW_SKEWED | KW_ROLLUP | KW_CUBE | KW_DIRECTORIES | KW_FOR | KW_GROUPING | KW_SETS | KW_TRUNCATE | KW_NOSCAN | KW_USER | KW_ROLE | KW_ROLES | KW_INNER | KW_DEFINED | KW_ADMIN | KW_JAR | KW_FILE | KW_OWNER | KW_PRINCIPALS | KW_ALL | KW_DEFAULT | KW_NONE
+    KW_TRUE | KW_FALSE | KW_LIKE | KW_EXISTS | KW_ASC | KW_DESC | KW_ORDER | KW_GROUP | KW_BY | KW_AS | KW_INSERT | KW_OVERWRITE | KW_OUTER | KW_LEFT | KW_RIGHT | KW_FULL | KW_PARTITION | KW_PARTITIONS | KW_TABLE | KW_TABLES | KW_COLUMNS | KW_INDEX | KW_INDEXES | KW_REBUILD | KW_FUNCTIONS | KW_SHOW | KW_MSCK | KW_REPAIR | KW_DIRECTORY | KW_LOCAL | KW_USING | KW_CLUSTER | KW_DISTRIBUTE | KW_SORT | KW_UNION | KW_LOAD | KW_EXPORT | KW_IMPORT | KW_DATA | KW_INPATH | KW_IS | KW_NULL | KW_CREATE | KW_EXTERNAL | KW_ALTER | KW_CHANGE | KW_FIRST | KW_AFTER | KW_DESCRIBE | KW_DROP | KW_RENAME | KW_IGNORE | KW_PROTECTION | KW_TO | KW_COMMENT | KW_BOOLEAN | KW_TINYINT | KW_SMALLINT | KW_INT | KW_BIGINT | KW_FLOAT | KW_DOUBLE | KW_DATE | KW_DATETIME | KW_TIMESTAMP | KW_DECIMAL | KW_STRING | KW_ARRAY | KW_STRUCT | KW_UNIONTYPE | KW_PARTITIONED | KW_CLUSTERED | KW_SORTED | KW_INTO | KW_BUCKETS | KW_ROW | KW_ROWS | KW_FORMAT | KW_DELIMITED | KW_FIELDS | KW_TERMINATED | KW_ESCAPED | KW_COLLECTION | 
 KW_ITEMS | KW_KEYS | KW_KEY_TYPE | KW_LINES | KW_STORED | KW_FILEFORMAT | KW_SEQUENCEFILE | KW_TEXTFILE | KW_RCFILE | KW_ORCFILE | KW_PARQUETFILE | KW_INPUTFORMAT | KW_OUTPUTFORMAT | KW_INPUTDRIVER | KW_OUTPUTDRIVER | KW_OFFLINE | KW_ENABLE | KW_DISABLE | KW_READONLY | KW_NO_DROP | KW_LOCATION | KW_BUCKET | KW_OUT | KW_OF | KW_PERCENT | KW_ADD | KW_REPLACE | KW_RLIKE | KW_REGEXP | KW_TEMPORARY | KW_EXPLAIN | KW_FORMATTED | KW_PRETTY | KW_DEPENDENCY | KW_LOGICAL | KW_SERDE | KW_WITH | KW_DEFERRED | KW_SERDEPROPERTIES | KW_DBPROPERTIES | KW_LIMIT | KW_SET | KW_UNSET | KW_TBLPROPERTIES | KW_IDXPROPERTIES | KW_VALUE_TYPE | KW_ELEM_TYPE | KW_MAPJOIN | KW_STREAMTABLE | KW_HOLD_DDLTIME | KW_CLUSTERSTATUS | KW_UTC | KW_UTCTIMESTAMP | KW_LONG | KW_DELETE | KW_PLUS | KW_MINUS | KW_FETCH | KW_INTERSECT | KW_VIEW | KW_IN | KW_DATABASES | KW_MATERIALIZED | KW_SCHEMA | KW_SCHEMAS | KW_GRANT | KW_REVOKE | KW_SSL | KW_UNDO | KW_LOCK | KW_LOCKS | KW_UNLOCK | KW_SHARED | KW_EXCLUSIVE | KW_PROCEDURE |
  KW_UNSIGNED | KW_WHILE | KW_READ | KW_READS | KW_PURGE | KW_RANGE | KW_ANALYZE | KW_BEFORE | KW_BETWEEN | KW_BOTH | KW_BINARY | KW_CONTINUE | KW_CURSOR | KW_TRIGGER | KW_RECORDREADER | KW_RECORDWRITER | KW_SEMI | KW_LATERAL | KW_TOUCH | KW_ARCHIVE | KW_UNARCHIVE | KW_COMPUTE | KW_STATISTICS | KW_USE | KW_OPTION | KW_CONCATENATE | KW_SHOW_DATABASE | KW_UPDATE | KW_RESTRICT | KW_CASCADE | KW_SKEWED | KW_ROLLUP | KW_CUBE | KW_DIRECTORIES | KW_FOR | KW_GROUPING | KW_SETS | KW_TRUNCATE | KW_NOSCAN | KW_USER | KW_ROLE | KW_ROLES | KW_INNER | KW_DEFINED | KW_ADMIN | KW_JAR | KW_FILE | KW_OWNER | KW_PRINCIPALS | KW_ALL | KW_DEFAULT | KW_NONE | KW_COMPACT | KW_COMPACTIONS | KW_TRANSACTIONS
     ;

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/parse/SemanticAnalyzerFactory.java Tue Mar 25 22:14:52 2014
@@ -106,6 +106,8 @@ public final class SemanticAnalyzerFacto
     commandType.put(HiveParser.TOK_ANALYZE, HiveOperation.ANALYZE_TABLE);
     commandType.put(HiveParser.TOK_ALTERVIEW_RENAME, HiveOperation.ALTERVIEW_RENAME);
     commandType.put(HiveParser.TOK_ALTERTABLE_PARTCOLTYPE, HiveOperation.ALTERTABLE_PARTCOLTYPE);
+    commandType.put(HiveParser.TOK_SHOW_COMPACTIONS, HiveOperation.SHOW_COMPACTIONS);
+    commandType.put(HiveParser.TOK_SHOW_TRANSACTIONS, HiveOperation.SHOW_TRANSACTIONS);
   }
 
   static {
@@ -130,6 +132,8 @@ public final class SemanticAnalyzerFacto
             HiveOperation.ALTERPARTITION_SERDEPROPERTIES });
     tablePartitionCommandType.put(HiveParser.TOK_ALTERTABLE_RENAMEPART,
         new HiveOperation[] {null, HiveOperation.ALTERTABLE_RENAMEPART});
+    tablePartitionCommandType.put(HiveParser.TOK_COMPACT,
+        new HiveOperation[] {null, HiveOperation.ALTERTABLE_COMPACT});
     tablePartitionCommandType.put(HiveParser.TOK_ALTERTBLPART_SKEWED_LOCATION,
         new HiveOperation[] {HiveOperation.ALTERTBLPART_SKEWED_LOCATION,
             HiveOperation.ALTERTBLPART_SKEWED_LOCATION });
@@ -195,6 +199,8 @@ public final class SemanticAnalyzerFacto
       case HiveParser.TOK_SHOWINDEXES:
       case HiveParser.TOK_SHOWLOCKS:
       case HiveParser.TOK_SHOWDBLOCKS:
+      case HiveParser.TOK_SHOW_COMPACTIONS:
+      case HiveParser.TOK_SHOW_TRANSACTIONS:
       case HiveParser.TOK_CREATEINDEX:
       case HiveParser.TOK_DROPINDEX:
       case HiveParser.TOK_ALTERTABLE_CLUSTER_SORT:

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableDesc.java Tue Mar 25 22:14:52 2014
@@ -48,7 +48,7 @@ public class AlterTableDesc extends DDLD
     ADDFILEFORMAT, ADDCLUSTERSORTCOLUMN, RENAMECOLUMN, ADDPARTITION,
     TOUCH, ARCHIVE, UNARCHIVE, ALTERPROTECTMODE, ALTERPARTITIONPROTECTMODE,
     ALTERLOCATION, DROPPARTITION, RENAMEPARTITION, ADDSKEWEDBY, ALTERSKEWEDLOCATION,
-    ALTERBUCKETNUM, ALTERPARTITION
+    ALTERBUCKETNUM, ALTERPARTITION, COMPACT
   }
 
   public static enum ProtectModeType {
@@ -702,4 +702,5 @@ public class AlterTableDesc extends DDLD
     return isDropIfExists;
   }
 
+
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/AlterTableSimpleDesc.java Tue Mar 25 22:14:52 2014
@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.hive.ql.plan;
 
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.Map;
 
@@ -30,6 +31,7 @@ public class AlterTableSimpleDesc extend
   private String tableName;
   private String dbName;
   private LinkedHashMap<String, String> partSpec;
+  private String compactionType;
 
   AlterTableTypes type;
 
@@ -57,6 +59,22 @@ public class AlterTableSimpleDesc extend
     this.type = type;
   }
 
+  /**
+   * Constructor for ALTER TABLE ... COMPACT.
+   * @param dbname name of the database containing the table
+   * @param tableName name of the table to compact
+   * @param partSpec partition to compact
+   * @param compactionType currently supported values: 'major' and 'minor'
+   */
+  public AlterTableSimpleDesc(String dbname, String tableName,
+                              LinkedHashMap<String,  String> partSpec,  String compactionType) {
+    type = AlterTableTypes.COMPACT;
+    this.compactionType = compactionType;
+    this.dbName = dbname;
+    this.tableName = tableName;
+    this.partSpec = partSpec;
+  }
+
   public String getTableName() {
     return tableName;
   }
@@ -89,4 +107,12 @@ public class AlterTableSimpleDesc extend
     this.partSpec = partSpec;
   }
 
+  /**
+   * Get what type of compaction is being done by a ALTER TABLE ... COMPACT statement.
+   * @return Compaction type, currently supported values are 'major' and 'minor'.
+   */
+  public String getCompactionType() {
+    return compactionType;
+  }
+
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/DDLWork.java Tue Mar 25 22:14:52 2014
@@ -52,6 +52,8 @@ public class DDLWork implements Serializ
   private UnlockTableDesc unlockTblDesc;
   private ShowFunctionsDesc showFuncsDesc;
   private ShowLocksDesc showLocksDesc;
+  private ShowCompactionsDesc showCompactionsDesc;
+  private ShowTxnsDesc showTxnsDesc;
   private DescFunctionDesc descFunctionDesc;
   private ShowPartitionsDesc showPartsDesc;
   private ShowCreateTableDesc showCreateTblDesc;
@@ -323,7 +325,19 @@ public class DDLWork implements Serializ
     this.showLocksDesc = showLocksDesc;
   }
 
-  /**
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+                 ShowCompactionsDesc showCompactionsDesc) {
+    this(inputs, outputs);
+    this.showCompactionsDesc = showCompactionsDesc;
+  }
+
+  public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
+                 ShowTxnsDesc showTxnsDesc) {
+    this(inputs, outputs);
+    this.showTxnsDesc = showTxnsDesc;
+  }
+
+   /**
    * @param descFuncDesc
    */
   public DDLWork(HashSet<ReadEntity> inputs, HashSet<WriteEntity> outputs,
@@ -711,6 +725,16 @@ public class DDLWork implements Serializ
     return showLocksDesc;
   }
 
+  @Explain(displayName = "Show Compactions Operator")
+  public ShowCompactionsDesc getShowCompactionsDesc() {
+    return showCompactionsDesc;
+  }
+
+  @Explain(displayName = "Show Transactions Operator")
+  public ShowTxnsDesc getShowTxnsDesc() {
+    return showTxnsDesc;
+  }
+
   /**
    * @return the lockTblDesc
    */
@@ -751,6 +775,14 @@ public class DDLWork implements Serializ
     this.showLocksDesc = showLocksDesc;
   }
 
+  public void setShowCompactionsDesc(ShowCompactionsDesc showCompactionsDesc) {
+    this.showCompactionsDesc = showCompactionsDesc;
+  }
+
+  public void setShowTxnsDesc(ShowTxnsDesc showTxnsDesc) {
+    this.showTxnsDesc = showTxnsDesc;
+  }
+
   /**
    * @param lockTblDesc
    *          the lockTblDesc to set

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/HiveOperation.java Tue Mar 25 22:14:52 2014
@@ -111,6 +111,10 @@ public enum HiveOperation {
       new Privilege[] {Privilege.ALTER_DATA}, null),
   ALTERTABLE_PARTCOLTYPE("ALTERTABLE_PARTCOLTYPE", new Privilege[] { Privilege.SELECT }, new Privilege[] { Privilege.ALTER_DATA }),
   ALTERVIEW_RENAME("ALTERVIEW_RENAME", new Privilege[] {Privilege.ALTER_METADATA}, null),
+  ALTERTABLE_COMPACT("ALTERTABLE_COMPACT", new Privilege[]{Privilege.SELECT},
+      new Privilege[]{Privilege.ALTER_DATA}),
+  SHOW_COMPACTIONS("SHOW COMPACTIONS", null, null),
+  SHOW_TRANSACTIONS("SHOW TRANSACTIONS", null, null);
   ;
 
   private String operationName;

Added: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCompactionsDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCompactionsDesc.java?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCompactionsDesc.java (added)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowCompactionsDesc.java Tue Mar 25 22:14:52 2014
@@ -0,0 +1,56 @@
+/**
+ * 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 org.apache.hadoop.fs.Path;
+
+import java.io.Serializable;
+
+/**
+ * Descriptor for showing compactions.
+ */
+public class ShowCompactionsDesc extends DDLDesc implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  private static final String schema = "dbname,tabname,partname,type,state,workerid," +
+      "starttime#string:string:string:string:string:string:string";
+
+  private String resFile;
+
+  /**
+   *
+   * @param resFile File that results of show will be written to.
+   */
+  public ShowCompactionsDesc(Path resFile) {
+    this.resFile = resFile.toString();
+  }
+
+  /**
+   *  No arg constructor for serialization.
+   */
+  public ShowCompactionsDesc() {
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public String getResFile() {
+    return resFile;
+  }
+}

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowLocksDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowLocksDesc.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowLocksDesc.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowLocksDesc.java Tue Mar 25 22:14:52 2014
@@ -22,6 +22,7 @@ import java.io.Serializable;
 import java.util.HashMap;
 
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
 
 /**
  * ShowLocksDesc.
@@ -35,6 +36,7 @@ public class ShowLocksDesc extends DDLDe
   String tableName;
   HashMap<String, String> partSpec;
   boolean isExt;
+  boolean isNewLockFormat;
 
   /**
    * table name for the result of show locks.
@@ -45,6 +47,13 @@ public class ShowLocksDesc extends DDLDe
    */
   private static final String schema = "tab_name,mode#string:string";
 
+  /**
+   * Schema for use with db txn manager.
+   */
+  private static final String newFormatSchema = "lockid,database,table,partition,lock_state," +
+      "lock_type,transaction_id,last_heartbeat,acquired_at,user," +
+      "hostname#string:string:string:string:string:string:string:string:string:string:string";
+
   public String getDatabase() {
     return dbName;
   }
@@ -54,7 +63,8 @@ public class ShowLocksDesc extends DDLDe
   }
 
   public String getSchema() {
-    return schema;
+    if (isNewLockFormat) return newFormatSchema;
+    else return schema;
   }
 
   public ShowLocksDesc() {
@@ -63,23 +73,25 @@ public class ShowLocksDesc extends DDLDe
   /**
    * @param resFile
    */
-  public ShowLocksDesc(Path resFile, String dbName, boolean isExt) {
+  public ShowLocksDesc(Path resFile, String dbName, boolean isExt, boolean isNewFormat) {
     this.resFile   = resFile.toString();
     this.partSpec  = null;
     this.tableName = null;
     this.isExt     = isExt;
     this.dbName = dbName;
+    isNewLockFormat = isNewFormat;
   }
 
   /**
    * @param resFile
    */
   public ShowLocksDesc(Path resFile, String tableName,
-                       HashMap<String, String> partSpec, boolean isExt) {
+                       HashMap<String, String> partSpec, boolean isExt, boolean isNewFormat) {
     this.resFile   = resFile.toString();
     this.partSpec  = partSpec;
     this.tableName = tableName;
     this.isExt     = isExt;
+    isNewLockFormat = isNewFormat;
   }
 
   public String getDbName() {
@@ -152,4 +164,8 @@ public class ShowLocksDesc extends DDLDe
   public void setExt(boolean isExt) {
     this.isExt = isExt;
   }
+
+  public boolean isNewFormat() {
+    return isNewLockFormat;
+  }
 }

Added: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTxnsDesc.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTxnsDesc.java?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTxnsDesc.java (added)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/plan/ShowTxnsDesc.java Tue Mar 25 22:14:52 2014
@@ -0,0 +1,55 @@
+/**
+ * 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 org.apache.hadoop.fs.Path;
+
+import java.io.Serializable;
+
+/**
+ * Descriptor for showing transactions.
+ */
+public class ShowTxnsDesc extends DDLDesc implements Serializable {
+
+  private static final long serialVersionUID = 1L;
+  private static final String schema = "txnid,state,user,host#string:string:string:string";
+
+  private String resFile;
+
+  /**
+   *
+   * @param resFile File that results of show will be written to.
+   */
+  public ShowTxnsDesc(Path resFile) {
+    this.resFile = resFile.toString();
+  }
+
+  /**
+   *  No arg constructor for serialization.
+   */
+  public ShowTxnsDesc() {
+  }
+
+  public String getSchema() {
+    return schema;
+  }
+
+  public String getResFile() {
+    return resFile;
+  }
+}

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/HiveOperationType.java Tue Mar 25 22:14:52 2014
@@ -111,6 +111,9 @@ public enum HiveOperationType {
   ALTERTABLE_SKEWED,
   ALTERTBLPART_SKEWED_LOCATION,
   ALTERVIEW_RENAME,
+  ALTERTABLE_COMPACT,
+  SHOW_COMPACTIONS,
+  SHOW_TRANSACTIONS
 
 
 }

Modified: hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java?rev=1581559&r1=1581558&r2=1581559&view=diff
==============================================================================
--- hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java (original)
+++ hive/branches/branch-0.13/ql/src/java/org/apache/hadoop/hive/ql/security/authorization/plugin/sqlstd/Operation2Privilege.java Tue Mar 25 22:14:52 2014
@@ -117,6 +117,7 @@ public class Operation2Privilege {
     op2Priv.put(HiveOperationType.ALTERPARTITION_MERGEFILES, new InOutPrivs(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTABLE_SKEWED, new InOutPrivs(OWNER_PRIV_AR, OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.ALTERTBLPART_SKEWED_LOCATION, new InOutPrivs(OWNER_PRIV_AR, OWNER_INS_SEL_DEL_NOGRANT_AR));
+    op2Priv.put(HiveOperationType.ALTERTABLE_COMPACT, new InOutPrivs(OWNER_PRIV_AR,  OWNER_PRIV_AR));
     op2Priv.put(HiveOperationType.TRUNCATETABLE, new InOutPrivs(OWNER_PRIV_AR, OWNER_PRIV_AR));
 
     //table ownership for create/drop/alter index
@@ -173,6 +174,9 @@ public class Operation2Privilege {
     op2Priv.put(HiveOperationType.DROPFUNCTION, new InOutPrivs(null, ADMIN_PRIV_AR));
     op2Priv.put(HiveOperationType.CREATEMACRO, new InOutPrivs(null, ADMIN_PRIV_AR));
     op2Priv.put(HiveOperationType.DROPMACRO, new InOutPrivs(null, ADMIN_PRIV_AR));
+    op2Priv.put(HiveOperationType.SHOW_COMPACTIONS, new InOutPrivs(null, null));
+    op2Priv.put(HiveOperationType.SHOW_TRANSACTIONS, new InOutPrivs(null, null));
+    op2Priv.put(HiveOperationType.DROPFUNCTION, new InOutPrivs(null, null));
 
     op2Priv.put(HiveOperationType.LOCKTABLE, new InOutPrivs(null, null));
     op2Priv.put(HiveOperationType.UNLOCKTABLE, new InOutPrivs(null, null));

Added: hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java (added)
+++ hive/branches/branch-0.13/ql/src/test/org/apache/hadoop/hive/ql/parse/TestQBCompact.java Tue Mar 25 22:14:52 2014
@@ -0,0 +1,131 @@
+/**
+ * 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.parse;
+
+import junit.framework.Assert;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.hadoop.hive.ql.Context;
+import org.apache.hadoop.hive.ql.ErrorMsg;
+import org.apache.hadoop.hive.ql.exec.Task;
+import org.apache.hadoop.hive.ql.io.orc.OrcInputFormat;
+import org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat;
+import org.apache.hadoop.hive.ql.metadata.Hive;
+import org.apache.hadoop.hive.ql.metadata.Table;
+import org.apache.hadoop.hive.ql.plan.AlterTableSimpleDesc;
+import org.apache.hadoop.hive.ql.plan.DDLWork;
+import org.apache.hadoop.hive.ql.session.SessionState;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Tests for parsing and semantic analysis of ALTER TABLE ... compact.
+ */
+public class TestQBCompact {
+  static HiveConf conf;
+
+  @BeforeClass
+  public static void init() throws Exception {
+    conf = new HiveConf();
+    SessionState.start(conf);
+
+    // Create a table so we can work against it
+    Hive h = Hive.get(conf);
+    List<String> cols = new ArrayList<String>();
+    cols.add("a");
+    List<String> partCols = new ArrayList<String>();
+    partCols.add("ds");
+    h.createTable("foo", cols, partCols, OrcInputFormat.class, OrcOutputFormat.class);
+    Table t = h.getTable("foo");
+    Map<String, String> partSpec = new HashMap<String, String>();
+    partSpec.put("ds", "today");
+    h.createPartition(t, partSpec);
+  }
+
+  private AlterTableSimpleDesc parseAndAnalyze(String query) throws Exception {
+    ParseDriver hd = new ParseDriver();
+    ASTNode head = (ASTNode)hd.parse(query).getChild(0);
+    System.out.println("HERE " + head.dump());
+    BaseSemanticAnalyzer a = SemanticAnalyzerFactory.get(conf, head);
+    a.analyze(head, new Context(conf));
+    List<Task<? extends Serializable>> roots = a.getRootTasks();
+    Assert.assertEquals(1, roots.size());
+    return ((DDLWork)roots.get(0).getWork()).getAlterTblSimpleDesc();
+  }
+
+
+  @Test
+  public void testNonPartitionedTable() throws Exception {
+    boolean sawException = false;
+    AlterTableSimpleDesc desc = parseAndAnalyze("alter table foo compact 'major'");
+    Assert.assertEquals("major", desc.getCompactionType());
+    Assert.assertEquals("foo", desc.getTableName());
+    Assert.assertEquals("default", desc.getDbName());
+  }
+
+  @Test
+  public void testBogusLevel() throws Exception {
+    boolean sawException = false;
+    try {
+      parseAndAnalyze("alter table foo partition(ds = 'today') compact 'bogus'");
+    } catch (SemanticException e) {
+      sawException = true;
+      Assert.assertEquals(ErrorMsg.INVALID_COMPACTION_TYPE.getMsg(), e.getMessage());
+    }
+    Assert.assertTrue(sawException);
+  }
+
+  @Test
+  public void testMajor() throws Exception {
+    AlterTableSimpleDesc desc =
+        parseAndAnalyze("alter table foo partition(ds = 'today') compact 'major'");
+    Assert.assertEquals("major", desc.getCompactionType());
+    Assert.assertEquals("foo", desc.getTableName());
+    Assert.assertEquals("default", desc.getDbName());
+    HashMap<String, String> parts = desc.getPartSpec();
+    Assert.assertEquals(1, parts.size());
+    Assert.assertEquals("today", parts.get("ds"));
+  }
+
+  @Test
+  public void testMinor() throws Exception {
+    AlterTableSimpleDesc desc =
+        parseAndAnalyze("alter table foo partition(ds = 'today') compact 'minor'");
+    Assert.assertEquals("minor", desc.getCompactionType());
+    Assert.assertEquals("foo", desc.getTableName());
+    Assert.assertEquals("default", desc.getDbName());
+    HashMap<String, String> parts = desc.getPartSpec();
+    Assert.assertEquals(1, parts.size());
+    Assert.assertEquals("today", parts.get("ds"));
+  }
+
+  @Test
+  public void showCompactions() throws Exception {
+    parseAndAnalyze("show compactions");
+  }
+
+  @Test
+  public void showTxns() throws Exception {
+    parseAndAnalyze("show transactions");
+  }
+}

Added: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q (added)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact1.q Tue Mar 25 22:14:52 2014
@@ -0,0 +1,12 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create table T1(key string, val string) stored as textfile;
+
+set hive.txn.testing=true;
+alter table T1 compact 'major';
+
+alter table T1 compact 'minor';
+
+drop table T1;

Added: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q (added)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact2.q Tue Mar 25 22:14:52 2014
@@ -0,0 +1,14 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create table T1(key string, val string) partitioned by (ds string) stored as textfile;
+
+alter table T1 add partition (ds = 'today');
+alter table T1 add partition (ds = 'yesterday');
+
+alter table T1 partition (ds = 'today') compact 'major';
+
+alter table T1 partition (ds = 'yesterday') compact 'minor';
+
+drop table T1;

Added: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q (added)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_compact3.q Tue Mar 25 22:14:52 2014
@@ -0,0 +1,15 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+create database D1;
+
+use D1;
+
+create table T1(key string, val string) stored as textfile;
+
+alter table T1 compact 'major';
+
+alter table T1 compact 'minor';
+
+drop table T1;

Added: hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q (added)
+++ hive/branches/branch-0.13/ql/src/test/queries/clientpositive/dbtxnmgr_showlocks.q Tue Mar 25 22:14:52 2014
@@ -0,0 +1,11 @@
+set hive.support.concurrency=true;
+set hive.txn.manager=org.apache.hadoop.hive.ql.lockmgr.DbTxnManager;
+set hive.txn.testing=true;
+
+show locks;
+
+show locks extended;
+
+show locks default;
+
+show transactions;

Added: hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact1.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact1.q.out?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact1.q.out (added)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact1.q.out Tue Mar 25 22:14:52 2014
@@ -0,0 +1,23 @@
+PREHOOK: query: create table T1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: create table T1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@T1
+PREHOOK: query: alter table T1 compact 'major'
+PREHOOK: type: null
+POSTHOOK: query: alter table T1 compact 'major'
+POSTHOOK: type: null
+PREHOOK: query: alter table T1 compact 'minor'
+PREHOOK: type: null
+POSTHOOK: query: alter table T1 compact 'minor'
+POSTHOOK: type: null
+PREHOOK: query: drop table T1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+POSTHOOK: query: drop table T1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1

Added: hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact2.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact2.q.out?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact2.q.out (added)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact2.q.out Tue Mar 25 22:14:52 2014
@@ -0,0 +1,37 @@
+PREHOOK: query: create table T1(key string, val string) partitioned by (ds string) stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+POSTHOOK: query: create table T1(key string, val string) partitioned by (ds string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@T1
+PREHOOK: query: alter table T1 add partition (ds = 'today')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@t1
+POSTHOOK: query: alter table T1 add partition (ds = 'today')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1@ds=today
+PREHOOK: query: alter table T1 add partition (ds = 'yesterday')
+PREHOOK: type: ALTERTABLE_ADDPARTS
+PREHOOK: Output: default@t1
+POSTHOOK: query: alter table T1 add partition (ds = 'yesterday')
+POSTHOOK: type: ALTERTABLE_ADDPARTS
+POSTHOOK: Output: default@t1
+POSTHOOK: Output: default@t1@ds=yesterday
+PREHOOK: query: alter table T1 partition (ds = 'today') compact 'major'
+PREHOOK: type: ALTERTABLE_COMPACT
+POSTHOOK: query: alter table T1 partition (ds = 'today') compact 'major'
+POSTHOOK: type: ALTERTABLE_COMPACT
+PREHOOK: query: alter table T1 partition (ds = 'yesterday') compact 'minor'
+PREHOOK: type: ALTERTABLE_COMPACT
+POSTHOOK: query: alter table T1 partition (ds = 'yesterday') compact 'minor'
+POSTHOOK: type: ALTERTABLE_COMPACT
+PREHOOK: query: drop table T1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@t1
+PREHOOK: Output: default@t1
+POSTHOOK: query: drop table T1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@t1
+POSTHOOK: Output: default@t1

Added: hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact3.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact3.q.out?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact3.q.out (added)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_compact3.q.out Tue Mar 25 22:14:52 2014
@@ -0,0 +1,31 @@
+PREHOOK: query: create database D1
+PREHOOK: type: CREATEDATABASE
+POSTHOOK: query: create database D1
+POSTHOOK: type: CREATEDATABASE
+PREHOOK: query: use D1
+PREHOOK: type: SWITCHDATABASE
+POSTHOOK: query: use D1
+POSTHOOK: type: SWITCHDATABASE
+PREHOOK: query: create table T1(key string, val string) stored as textfile
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:d1
+POSTHOOK: query: create table T1(key string, val string) stored as textfile
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: D1@T1
+POSTHOOK: Output: database:d1
+PREHOOK: query: alter table T1 compact 'major'
+PREHOOK: type: null
+POSTHOOK: query: alter table T1 compact 'major'
+POSTHOOK: type: null
+PREHOOK: query: alter table T1 compact 'minor'
+PREHOOK: type: null
+POSTHOOK: query: alter table T1 compact 'minor'
+POSTHOOK: type: null
+PREHOOK: query: drop table T1
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: d1@t1
+PREHOOK: Output: d1@t1
+POSTHOOK: query: drop table T1
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: d1@t1
+POSTHOOK: Output: d1@t1

Added: hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_showlocks.q.out
URL: http://svn.apache.org/viewvc/hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_showlocks.q.out?rev=1581559&view=auto
==============================================================================
--- hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_showlocks.q.out (added)
+++ hive/branches/branch-0.13/ql/src/test/results/clientpositive/dbtxnmgr_showlocks.q.out Tue Mar 25 22:14:52 2014
@@ -0,0 +1,20 @@
+PREHOOK: query: show locks
+PREHOOK: type: SHOWLOCKS
+POSTHOOK: query: show locks
+POSTHOOK: type: SHOWLOCKS
+Lock ID	Database	Table	Partition	State	Type	Transaction ID	Last Hearbeat	Acquired At	User	Hostname
+PREHOOK: query: show locks extended
+PREHOOK: type: SHOWLOCKS
+POSTHOOK: query: show locks extended
+POSTHOOK: type: SHOWLOCKS
+Lock ID	Database	Table	Partition	State	Type	Transaction ID	Last Hearbeat	Acquired At	User	Hostname
+PREHOOK: query: show locks default
+PREHOOK: type: SHOWLOCKS
+POSTHOOK: query: show locks default
+POSTHOOK: type: SHOWLOCKS
+Lock ID	Database	Table	Partition	State	Type	Transaction ID	Last Hearbeat	Acquired At	User	Hostname
+PREHOOK: query: show transactions
+PREHOOK: type: SHOW TRANSACTIONS
+POSTHOOK: query: show transactions
+POSTHOOK: type: SHOW TRANSACTIONS
+Transaction ID	Transaction State	User	Hostname