You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/03/16 09:12:21 UTC

[GitHub] [hive] kishendas commented on a change in pull request #2073: HIVE-24828: Provide new HMS API to return latest committed compaction…

kishendas commented on a change in pull request #2073:
URL: https://github.com/apache/hive/pull/2073#discussion_r594555075



##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/IMetaStoreClient.java
##########
@@ -3468,6 +3468,15 @@ CompactionResponse compact2(String dbname, String tableName, String partitionNam
    */
   ShowCompactResponse showCompactions() throws TException;
 
+  /**
+   * Get one latest record of completed compaction by given table/partition.
+   * @param request info on which compaction to retrieve
+   * @return one completed compaction record or null
+   * @throws TException
+   */
+  GetLatestCompactionResponse getLatestCompaction(GetLatestCompactionRequest request)

Review comment:
       Should it be more like getLatestCompactionRecord or getLatestCompactionInfo ?

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -255,6 +235,109 @@ public void testShowCompactions() throws Exception {
     });
   }
 
+  @Test
+  public void testGetLatestCompaction() throws Exception {
+    final String dbName = "foo";
+    final String tableName = "bar";
+    final String errorMessage = "Dummy error";
+    addSucceededCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    addFailedCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tableName);
+    GetLatestCompactionResponse response = txnHandler.getLatestCompaction(rqst);
+
+    assertNotNull("Should got a response", response);

Review comment:
       "Non-null response" sounds better

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -18,27 +18,7 @@
 package org.apache.hadoop.hive.metastore.txn;
 
 import org.apache.hadoop.hive.conf.HiveConf;
-import org.apache.hadoop.hive.metastore.api.AbortTxnRequest;
-import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions;
-import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest;
-import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse;
-import org.apache.hadoop.hive.metastore.api.CommitTxnRequest;
-import org.apache.hadoop.hive.metastore.api.CompactionRequest;
-import org.apache.hadoop.hive.metastore.api.CompactionType;
-import org.apache.hadoop.hive.metastore.api.DataOperationType;
-import org.apache.hadoop.hive.metastore.api.GetOpenTxnsResponse;
-import org.apache.hadoop.hive.metastore.api.LockComponent;
-import org.apache.hadoop.hive.metastore.api.LockLevel;
-import org.apache.hadoop.hive.metastore.api.LockRequest;
-import org.apache.hadoop.hive.metastore.api.LockResponse;
-import org.apache.hadoop.hive.metastore.api.LockState;
-import org.apache.hadoop.hive.metastore.api.LockType;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.OpenTxnRequest;
-import org.apache.hadoop.hive.metastore.api.OpenTxnsResponse;
-import org.apache.hadoop.hive.metastore.api.ShowCompactRequest;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponse;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
+import org.apache.hadoop.hive.metastore.api.*;

Review comment:
       We discourage use of .* . Please revert this change. 

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -255,6 +235,109 @@ public void testShowCompactions() throws Exception {
     });
   }
 
+  @Test
+  public void testGetLatestCompaction() throws Exception {
+    final String dbName = "foo";
+    final String tableName = "bar";
+    final String errorMessage = "Dummy error";
+    addSucceededCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    addFailedCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tableName);
+    GetLatestCompactionResponse response = txnHandler.getLatestCompaction(rqst);
+
+    assertNotNull("Should got a response", response);
+    assertEquals("Should got only one compaction", 1, response.getCompactionsSize());

Review comment:
       "Expecting a single compaction record" ?

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
##########
@@ -392,6 +387,39 @@ public void testGetValidWriteIds() throws TException {
     Assert.assertEquals(writeIdList.getMinOpenWriteId().longValue(), 2);
   }
 
+  @Test
+  public void testGetLatestCompaction() throws Exception {
+    final String dbName = "mydb";
+    final String tblName = "mytable";
+    Database db = new DatabaseBuilder().setName(dbName).build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    Table tbl = new TableBuilder().setDbName(dbName).setTableName(tblName)
+        .addCol("id", "int").addCol("name", "string")
+        .setType(TableType.MANAGED_TABLE.name()).build(conf);
+    client.createTable(tbl);
+    tbl = client.getTable(dbName, tblName);
+
+    client.compact2(tbl.getDbName(), tbl.getTableName(), null, CompactionType.MINOR, new HashMap<>());
+    OptionalCompactionInfoStruct optionalCi = client.findNextCompact("myworker");
+    client.markCleaned(optionalCi.getCi());
+
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tblName);
+    GetLatestCompactionResponse response = client.getLatestCompaction(rqst);
+
+    Assert.assertNotNull("Response should not be null", response);
+    Assert.assertEquals(1, response.getCompactionsSize());
+    Assert.assertEquals(dbName, response.getDbname());
+    Assert.assertEquals(tblName, response.getTablename());
+    LatestCompactionInfo lci = response.getCompactions().get(0);
+    Assert.assertEquals("Compaction ID should be 1", 1, lci.getId());

Review comment:
       Please remove all the redundant assert messages. 

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -331,14 +414,38 @@ private void addFailedCompaction(String dbName, String tableName, CompactionType
     CompactionRequest rqst;
     CompactionInfo ci;
     rqst = new CompactionRequest(dbName, tableName, type);
-    rqst.setPartitionname(partitionName);
+    if (partitionName != null) rqst.setPartitionname(partitionName);

Review comment:
       We use flower brackets, even for simple if statements. 
   Hope you are using the Apache Hive Java template in your IDE ?

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -255,6 +235,109 @@ public void testShowCompactions() throws Exception {
     });
   }
 
+  @Test
+  public void testGetLatestCompaction() throws Exception {
+    final String dbName = "foo";
+    final String tableName = "bar";
+    final String errorMessage = "Dummy error";
+    addSucceededCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    addFailedCompaction(dbName, tableName, CompactionType.MINOR, null, errorMessage);
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tableName);
+    GetLatestCompactionResponse response = txnHandler.getLatestCompaction(rqst);
+
+    assertNotNull("Should got a response", response);
+    assertEquals("Should got only one compaction", 1, response.getCompactionsSize());
+    LatestCompactionInfo lci = response.getCompactions().get(0);
+    assertEquals("Should return the first succeeded compaction", 1, lci.getId());
+    assertNull("Partitionname should be null for non partitioned table", lci.getPartitionname());
+    assertEquals(CompactionType.MINOR, lci.getType());
+  }
+
+  @Test
+  public void testGetLatestCompactionPartition() throws Exception {
+    final String dbName = "foo";
+    final String tableName = "bar";
+    final String partitionName = "ds=today";
+    final String errorMessage = "Dummy error";
+    addSucceededCompaction(dbName, tableName, CompactionType.MINOR, partitionName, errorMessage);
+    addFailedCompaction(dbName, tableName, CompactionType.MINOR, partitionName, errorMessage);
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tableName);
+    rqst.addToPartitionnames(partitionName);
+    GetLatestCompactionResponse response = txnHandler.getLatestCompaction(rqst);
+
+    assertNotNull("Should got a response", response);
+    assertEquals("Should got only one compaction", 1, response.getCompactionsSize());
+    LatestCompactionInfo lci = response.getCompactions().get(0);
+    assertEquals("Should return the first succeeded compaction", 1, lci.getId());
+    assertEquals("Should return same partition name as the request", partitionName, lci.getPartitionname());
+    assertEquals(CompactionType.MINOR, lci.getType());
+
+    final String anotherPartitionName = "ds=yesterday";
+    addWaitingForCleaningCompaction(dbName, tableName, CompactionType.MINOR, anotherPartitionName, errorMessage);
+    rqst.addToPartitionnames(anotherPartitionName);
+    response = txnHandler.getLatestCompaction(rqst);
+
+    assertNotNull("Should got a response", response);
+    assertEquals("Should got two compactions", 2, response.getCompactionsSize());

Review comment:
       "Expecting two compaction records" 

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3585,6 +3532,77 @@ public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaExcep
     }
   }
 
+  @RetrySemantics.ReadOnly
+  public GetLatestCompactionResponse getLatestCompaction(GetLatestCompactionRequest rqst) throws MetaException {
+    GetLatestCompactionResponse response = new GetLatestCompactionResponse();
+    response.setDbname(rqst.getDbname());
+    response.setTablename(rqst.getTablename());
+    Connection dbConn = null;
+    PreparedStatement pst = null;
+    try {
+      try {
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+
+        List<String> params = new ArrayList<>();
+        StringBuilder sb = new StringBuilder()

Review comment:
       Can you please add a comment on what this query is doing ?

##########
File path: ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java
##########
@@ -53,12 +33,12 @@
 import java.util.TreeSet;
 import java.util.concurrent.TimeUnit;
 
-import static junit.framework.Assert.assertEquals;
-import static junit.framework.Assert.assertFalse;
-import static junit.framework.Assert.assertNotNull;
-import static junit.framework.Assert.assertNull;

Review comment:
       Are you sure these changes in import is required ?

##########
File path: standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/TestHiveMetaStoreTxns.java
##########
@@ -392,6 +387,39 @@ public void testGetValidWriteIds() throws TException {
     Assert.assertEquals(writeIdList.getMinOpenWriteId().longValue(), 2);
   }
 
+  @Test
+  public void testGetLatestCompaction() throws Exception {
+    final String dbName = "mydb";
+    final String tblName = "mytable";
+    Database db = new DatabaseBuilder().setName(dbName).build(conf);
+    db.unsetCatalogName();
+    client.createDatabase(db);
+
+    Table tbl = new TableBuilder().setDbName(dbName).setTableName(tblName)
+        .addCol("id", "int").addCol("name", "string")
+        .setType(TableType.MANAGED_TABLE.name()).build(conf);
+    client.createTable(tbl);
+    tbl = client.getTable(dbName, tblName);
+
+    client.compact2(tbl.getDbName(), tbl.getTableName(), null, CompactionType.MINOR, new HashMap<>());
+    OptionalCompactionInfoStruct optionalCi = client.findNextCompact("myworker");
+    client.markCleaned(optionalCi.getCi());
+
+    GetLatestCompactionRequest rqst = new GetLatestCompactionRequest();
+    rqst.setDbname(dbName);
+    rqst.setTablename(tblName);
+    GetLatestCompactionResponse response = client.getLatestCompaction(rqst);
+
+    Assert.assertNotNull("Response should not be null", response);

Review comment:
       You don't have add a message, if it's self explanatory. 

##########
File path: standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java
##########
@@ -3585,6 +3532,77 @@ public ShowCompactResponse showCompact(ShowCompactRequest rqst) throws MetaExcep
     }
   }
 
+  @RetrySemantics.ReadOnly
+  public GetLatestCompactionResponse getLatestCompaction(GetLatestCompactionRequest rqst) throws MetaException {
+    GetLatestCompactionResponse response = new GetLatestCompactionResponse();
+    response.setDbname(rqst.getDbname());
+    response.setTablename(rqst.getTablename());
+    Connection dbConn = null;
+    PreparedStatement pst = null;
+    try {
+      try {
+        dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED);
+
+        List<String> params = new ArrayList<>();
+        StringBuilder sb = new StringBuilder()
+            .append("SELECT * FROM (")
+            .append("   SELECT")
+            .append("   \"CC_ID\", \"CC_DATABASE\", \"CC_TABLE\", \"CC_PARTITION\", \"CC_TYPE\"")
+            .append("   FROM \"COMPLETED_COMPACTIONS\"")
+            .append("     WHERE \"CC_STATE\" = " + quoteChar(SUCCEEDED_STATE))
+            .append("   UNION ALL")
+            .append("   SELECT")
+            .append("   \"CQ_ID\" AS \"CC_ID\", \"CQ_DATABASE\" AS \"CC_DATABASE\"")
+            .append("   ,\"CQ_TABLE\" AS \"CC_TABLE\", \"CQ_PARTITION\" AS \"CC_PARTITION\"")
+            .append("   ,\"CQ_TYPE\" AS \"CC_TYPE\"")
+            .append("   FROM \"COMPACTION_QUEUE\"")
+            .append("     WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING))
+            .append(") AS compactions ")
+            .append(" WHERE \"CC_DATABASE\"=? AND \"CC_TABLE\"=?");
+        params.add(rqst.getDbname());
+        params.add(rqst.getTablename());
+        if (rqst.getPartitionnames() != null) {
+          sb.append(" AND \"CC_PARTITION\" IN (");
+          sb.append(String.join(",",
+              Collections.nCopies(rqst.getPartitionnamesSize(), "?")));
+          sb.append(")");
+          params.addAll(rqst.getPartitionnames());
+        } else {
+          sb.append(" AND \"CC_PARTITION\" IS NULL");
+        }
+        sb.append(" ORDER BY \"CC_ID\" DESC");
+
+        pst = sqlGenerator.prepareStmtWithParameters(dbConn, sb.toString(), params);
+        LOG.debug("Going to execute query <" + sb.toString() + ">");
+        ResultSet rs = pst.executeQuery();
+        HashSet<String> partitionSet = new HashSet<>();
+        while (rs.next()) {
+          LatestCompactionInfo lci = new LatestCompactionInfo();
+          lci.setId(rs.getLong(1));
+          String partition = rs.getString(4);
+          if (!rs.wasNull()) lci.setPartitionname(partition);

Review comment:
       Please use flower brackets, even for simple if statements.

##########
File path: standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/utils/FilterUtils.java
##########
@@ -22,21 +22,10 @@
 import static org.apache.hadoop.hive.metastore.utils.MetaStoreUtils.CATALOG_DB_SEPARATOR;
 
 import org.apache.hadoop.hive.metastore.MetaStoreFilterHook;
-import org.apache.hadoop.hive.metastore.api.Catalog;
-import org.apache.hadoop.hive.metastore.api.Database;
-import org.apache.hadoop.hive.metastore.api.MetaException;
-import org.apache.hadoop.hive.metastore.api.NoSuchObjectException;
-import org.apache.hadoop.hive.metastore.api.Partition;
-import org.apache.hadoop.hive.metastore.api.PartitionSpec;
-import org.apache.hadoop.hive.metastore.api.Table;
-import org.apache.hadoop.hive.metastore.api.TableMeta;
-import org.apache.hadoop.hive.metastore.api.ShowCompactResponseElement;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
+import org.apache.hadoop.hive.metastore.api.*;
+
+import java.util.*;

Review comment:
       Please do not use .*;




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org