From ee686a9bd52167e569490defbbf1069161271200 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Tue, 25 Apr 2023 13:10:30 +0530 Subject: [PATCH 1/6] HIVE-27332: Add retry backoff mechanism for abort cleanup --- .../compactor/handler/AbortedTxnCleaner.java | 15 +- .../compactor/handler/CompactionCleaner.java | 22 +- .../ql/txn/compactor/handler/TaskHandler.java | 21 ++ .../hive/ql/txn/compactor/CompactorTest.java | 2 + .../hive/ql/txn/compactor/TestCleaner.java | 2 + .../handler/TestAbortedTxnCleaner.java | 309 ++++++++++++++++++ .../hive/metastore/conf/MetastoreConf.java | 2 +- .../metastore/txn/CompactionTxnHandler.java | 130 ++++++-- .../hadoop/hive/metastore/txn/TxnStore.java | 6 +- .../sql/derby/hive-schema-4.0.0.derby.sql | 9 + .../upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql | 10 + .../sql/mssql/hive-schema-4.0.0.mssql.sql | 9 + .../upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql | 10 + .../sql/mysql/hive-schema-4.0.0.mysql.sql | 9 + .../upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql | 10 + .../sql/oracle/hive-schema-4.0.0.oracle.sql | 9 + .../upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql | 10 + .../postgres/hive-schema-4.0.0.postgres.sql | 9 + ...pgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql | 10 + .../hive/metastore/utils/TestTxnDbUtil.java | 1 + 20 files changed, 558 insertions(+), 47 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index 06dc02942d90..26039ca4ca9d 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -41,6 +41,8 @@ import java.util.stream.Collectors; import static java.util.Objects.isNull; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME; +import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED; /** * Abort-cleanup based implementation of TaskHandler. @@ -63,7 +65,7 @@ aborted txns are not visible by any open txns. It is only visible while determin only sees the aborted deltas and does not read the file).

The following algorithm is used to clean the set of aborted directories -
- a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanAborts(long, int)}).
+ a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanAborts(long, int, long)}).
b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table.
c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID
d. Fetch the aborted directories and delete the directories.
@@ -76,7 +78,10 @@ public List getTasks() throws MetaException { long abortedTimeThreshold = HiveConf .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, TimeUnit.MILLISECONDS); - List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); + long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED) + ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS) + : 0; + List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold, retentionTime); if (!readyToCleanAborts.isEmpty()) { return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() -> @@ -118,10 +123,16 @@ private void clean(CompactionInfo info, long minOpenWriteTxn, boolean metricsEna abortCleanUsingAcidDir(info, location, minOpenWriteTxn); } catch (InterruptedException e) { + LOG.error("Caught an interrupted exception when cleaning, unable to complete cleaning of {} due to {}", info, + e.getMessage()); + info.errorMessage = e.getMessage(); + handleCleanerAttemptFailure(info, true); throw e; } catch (Exception e) { LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info, e.getMessage()); + info.errorMessage = e.getMessage(); + handleCleanerAttemptFailure(info, true); throw new MetaException(e.getMessage()); } finally { if (metricsEnabled) { diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java index f7c8944ea3ee..7920a53788a7 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java @@ -58,10 +58,6 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; import static java.util.Objects.isNull; /** @@ -172,7 +168,7 @@ private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) t if (metricsEnabled) { Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc(); } - handleCleanerAttemptFailure(ci); + handleCleanerAttemptFailure(ci, false); } finally { if (metricsEnabled) { perfLogger.perfLogEnd(CompactionCleaner.class.getName(), cleanerMetric); @@ -302,22 +298,6 @@ protected ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, V return validWriteIdList; } - private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException { - long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); - int cleanAttempts = 0; - if (ci.retryRetention > 0) { - cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1; - } - if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) { - //Mark it as failed if the max attempt threshold is reached. - txnHandler.markFailed(ci); - } else { - //Calculate retry retention time and update record. - ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; - txnHandler.setCleanerRetryRetentionTimeOnError(ci); - } - } - private CleanupRequest getCleaningRequestBasedOnLocation(CompactionInfo ci, String location) { String strIfPurge = ci.getProperty("ifPurge"); boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge")); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index ef95a100c1a2..3674c47910d1 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -49,8 +49,13 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.commons.collections.ListUtils.subtract; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; /** * An abstract class which defines the list of utility methods for performing cleanup activities. @@ -161,4 +166,20 @@ protected boolean cleanAndVerifyObsoleteDirectories(CompactionInfo info, String return success; } + + protected void handleCleanerAttemptFailure(CompactionInfo ci, boolean isAbort) throws MetaException { + long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); + int cleanAttempts = 0; + if (ci.retryRetention > 0) { + cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1; + } + if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS) && !isAbort) { + //Mark it as failed if the max attempt threshold is reached. + txnHandler.markFailed(ci); + } else { + //Calculate retry retention time and update record. + ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; + txnHandler.setCleanerRetryRetentionTimeOnError(ci, isAbort); + } + } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java index 113173acb37e..1899c989a15c 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java @@ -85,6 +85,7 @@ import org.apache.hive.common.util.HiveVersionInfo; import org.apache.thrift.TException; import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,6 +130,7 @@ public abstract class CompactorTest { FileSystem fs; @Before + @BeforeEach public void setup() throws Exception { setup(new HiveConf()); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java index df4a786a184b..01c9f09803f0 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java @@ -47,6 +47,7 @@ import org.junit.After; import org.junit.Assert; import org.junit.Test; +import org.junit.jupiter.api.AfterEach; import org.mockito.internal.util.reflection.FieldSetter; import java.util.ArrayList; @@ -811,6 +812,7 @@ boolean useHive130DeltaDirName() { } @After + @AfterEach public void tearDown() throws Exception { compactorTestCleanup(); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 3b40e2f75944..9eba02eafd16 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -25,6 +25,9 @@ import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.TxnStore; +import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.io.AcidUtils; import org.apache.hadoop.hive.ql.txn.compactor.Cleaner; import org.apache.hadoop.hive.ql.txn.compactor.CleanupRequest; @@ -32,14 +35,19 @@ import org.apache.hadoop.hive.ql.txn.compactor.MetadataCache; import org.junit.Assert; import org.junit.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; public class TestAbortedTxnCleaner extends TestHandler { @@ -320,4 +328,305 @@ public void testAbortedCleaningWithThreeTxnsWithDiffWriteIds() throws Exception List directories = getDirectories(conf, t, null); Assert.assertEquals(5, directories.size()); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_retry_entry", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TxnStore mockedTxnHandler = Mockito.spy(txnHandler); + TaskHandler mockedTaskHandler = Mockito.spy(new AbortedTxnCleaner(conf, mockedTxnHandler, metadataCache, + false, mockedFSRemover)); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); + cleaner.run(); + + Mockito.verify(mockedTxnHandler, Mockito.times(1)).setCleanerRetryRetentionTimeOnError(any(), eq(true)); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), + TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_retry_usage", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + // Set retry retention time as 10 s (10000 ms). + long retryRetentionTime = 10000; + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, retryRetentionTime, TimeUnit.MILLISECONDS); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler taskHandler = new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + + // Delay for time specified in retry retention. + Thread.sleep(retryRetentionTime); + + Mockito.doAnswer(InvocationOnMock::callRealMethod).when(mockedFSRemover).clean(any()); + + cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRetryWithNoDelay(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_retry_nodelay", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler taskHandler = new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), + TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + + Mockito.doAnswer(InvocationOnMock::callRealMethod).when(mockedFSRemover).clean(any()); + + cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + // The retry entry is not removed since retry conditions are not achieved hence its not picked for cleanup. + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), + TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_retry_retention_time_failed_twice", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + // Set retry retention time as 10 s (10000 ms). + long retryRetentionTime = 10000; + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, retryRetentionTime, TimeUnit.MILLISECONDS); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler taskHandler = new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + + // Delay for time specified in retry retention. + Thread.sleep(retryRetentionTime); + + cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + // The retry entry is not removed since retry has failed. + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + // The retry entry must reflect double the retention time now. + Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_retry_error_msg_failed_twice", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + // Set retry retention time as 10 s (10000 ms). + long retryRetentionTime = 10000; + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, retryRetentionTime, TimeUnit.MILLISECONDS); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler taskHandler = new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing first retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + + "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery) + .replace("\n", "").trim().contains("Testing first retry")); + + // Delay for time specified in retry retention. + Thread.sleep(retryRetentionTime); + + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing second retry"); + }).when(mockedFSRemover).clean(any()); + + cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + // The retry entry is not removed since retry has failed. + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + // The retry entry must reflect double the retention time now. + Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing second retry")); + } } diff --git a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java index 29c4b5774bea..112125c955dd 100644 --- a/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java +++ b/standalone-metastore/metastore-common/src/main/java/org/apache/hadoop/hive/metastore/conf/MetastoreConf.java @@ -651,7 +651,7 @@ public enum ConfVars { COMPACTOR_CLEANER_TABLECACHE_ON("metastore.compactor.cleaner.tablecache.on", "hive.compactor.cleaner.tablecache.on", true, "Enable table caching in the cleaner. Currently the cache is cleaned after each cycle."), - COMPACTOR_CLEAN_ABORTS_USING_CLEANER("metastore.compactor.clean.aborts.using.cleaner", "hive.compactor.clean.aborts.using.cleaner", false, + COMPACTOR_CLEAN_ABORTS_USING_CLEANER("metastore.compactor.clean.aborts.using.cleaner", "hive.compactor.clean.aborts.using.cleaner", true, "Whether to use cleaner for cleaning aborted directories or not.\n" + "Set to true when cleaner is expected to clean delta/delete-delta directories from aborted transactions.\n" + "Otherwise the cleanup of such directories will take place within the compaction cycle."), diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 83ceacbc30c5..750bb04a01d2 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -472,23 +472,38 @@ public List findReadyToClean(long minOpenTxnWaterMark, long rete @Override @RetrySemantics.ReadOnly - public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException { + public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold, long retentionTime) throws MetaException { try { List readyToCleanAborts = new ArrayList<>(); try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction); Statement stmt = dbConn.createStatement()) { boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0; - String sCheckAborted = "SELECT \"tc\".\"TC_DATABASE\", \"tc\".\"TC_TABLE\", \"tc\".\"TC_PARTITION\", " + - " \"tc\".\"MIN_TXN_START_TIME\", \"tc\".\"ABORTED_TXN_COUNT\", \"minOpenWriteTxnId\".\"MIN_OPEN_WRITE_TXNID\" FROM " + - " ( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " + - " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + - " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + - (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold) + " ) \"tc\" " + - " LEFT JOIN ( SELECT MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + - " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\"=" + TxnStatus.OPEN + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" ) \"minOpenWriteTxnId\" " + - " ON \"tc\".\"TC_DATABASE\" = \"minOpenWriteTxnId\".\"TC_DATABASE\" AND \"tc\".\"TC_TABLE\" = \"minOpenWriteTxnId\".\"TC_TABLE\"" + - " AND (\"tc\".\"TC_PARTITION\" = \"minOpenWriteTxnId\".\"TC_PARTITION\" OR (\"tc\".\"TC_PARTITION\" IS NULL AND \"minOpenWriteTxnId\".\"TC_PARTITION\" IS NULL))"; + String firstInnerQuery = "SELECT \"tc\".\"TC_DATABASE\" AS \"DB\", \"tc\".\"TC_TABLE\" AS \"TBL\", \"tc\".\"TC_PARTITION\" AS \"PART\", " + + " \"tc\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", \"tc\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", " + + " \"minOpenWriteTxnId\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", -1 AS \"RETRY_RETENTION\" FROM " + + " ( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " + + " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + + (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold) + " ) \"tc\" " + + " LEFT JOIN ( SELECT MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\"=" + TxnStatus.OPEN + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" ) \"minOpenWriteTxnId\" " + + " ON \"tc\".\"TC_DATABASE\" = \"minOpenWriteTxnId\".\"TC_DATABASE\" AND \"tc\".\"TC_TABLE\" = \"minOpenWriteTxnId\".\"TC_TABLE\"" + + " AND (\"tc\".\"TC_PARTITION\" = \"minOpenWriteTxnId\".\"TC_PARTITION\" OR (\"tc\".\"TC_PARTITION\" IS NULL AND \"minOpenWriteTxnId\".\"TC_PARTITION\" IS NULL))"; + + String secondInnerQuery = " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", " + + " 0 AS \"MIN_OPEN_WRITE_TXNID\", MAX(\"TCQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + " GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\" " + + " HAVING MIN(\"TCQ_RETRY_TIME\") < " + getEpochFn(dbProduct) + " - MAX(\"TCQ_RETRY_RETENTION\") - " + retentionTime; + + // Checks presence of the retry entry (by assigning 0 in retry retention) for the corresponding table/partition in the TXN_CLEANUP_QUEUE + String thirdInnerQuery = " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", " + + "0 AS \"MIN_OPEN_WRITE_TXNID\", 0 AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\""; + + String unionQuery = firstInnerQuery + " UNION ALL " + secondInnerQuery + " UNION ALL " + thirdInnerQuery; + + String sCheckAborted = "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\") " + + "FROM ( " + unionQuery + " ) res GROUP BY \"DB\", \"TBL\", \"PART\""; LOG.debug("Going to execute query <{}>", sCheckAborted); try (ResultSet rs = stmt.executeQuery(sCheckAborted)) { @@ -504,18 +519,28 @@ public List findReadyToCleanAborts(long abortedTimeThreshold, in info.partName = rs.getString(3); // In this case, this field contains min open write txn ID. info.txnId = rs.getLong(6); - readyToCleanAborts.add(info); + // The query returns the following retry retention values - 3 cases exist - + // -1 - No retry entry in TXN_CLEANUP_QUEUE + // 0 - Retry entry is present but hasn't achieved conditions for retry (i.e. min(tcq_retry_time) >= current_timestamp - max(retry_retention_time) - retentionTime) + // > 0 - Retry entry is present and has achieved conditions for retry (i.e. min(tcq_retry_time) < current_timestamp - max(retry_retention_time) - retentionTime) + info.retryRetention = rs.getLong(7); + // Do not add entry when retry retention is 0. + if (info.retryRetention != 0) { + // In this case, this flag is used for checking whether a retry entry needs to be inserted or updated. + info.hasOldAbort = info.retryRetention > 0; + readyToCleanAborts.add(info); + } } } } return readyToCleanAborts; } catch (SQLException e) { LOG.error("Unable to select next element for cleaning, " + e.getMessage()); - checkRetryable(e, "findReadyToCleanForAborts"); + checkRetryable(e, "findReadyToCleanAborts"); throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - return findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); + return findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold, retentionTime); } } @@ -708,6 +733,13 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws PreparedStatement pStmt = null; ResultSet rs = null; try { + /* + * Remove all retry associated metadata of table/partition in the TXN_CLEANUP_QUEUE both when compaction + * or abort cleanup is successful. We dont want a situation wherein we have a retry queue entry for a table + * but no corresponding entry in TXN_COMPONENTS table. + */ + removeRetryQueueEntries(dbConn, info); + /* * compaction may remove data from aborted txns above tc_writeid bit it only guarantees to * remove it up to (inclusive) tc_writeid, so it's critical to not remove metadata about @@ -773,6 +805,32 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws } } + private void removeRetryQueueEntries(Connection dbConn, CompactionInfo info) throws MetaException, RetryException { + PreparedStatement pStmt = null; + String query = "DELETE FROM \"TXN_CLEANUP_QUEUE\" WHERE \"TCQ_DATABASE\" = ? " + + "AND \"TCQ_TABLE\" = ? AND \"TCQ_PARTITION\" " + (info.partName != null ? "= ?" : "IS NULL"); + try { + LOG.debug("Going to execute update <{}>", query); + pStmt = dbConn.prepareStatement(query); + pStmt.setString(1, info.dbname); + pStmt.setString(2, info.tableName); + if (info.partName != null) { + pStmt.setString(3, info.partName); + } + int rc = pStmt.executeUpdate(); + LOG.debug("Removed {} records in txn_cleanup_queue", rc); + } catch (SQLException e) { + LOG.error("Unable to delete from txn components due to {}", e.getMessage()); + LOG.debug("Going to rollback"); + rollbackDBConn(dbConn); + checkRetryable(e, "removeRetryQueueEntries(" + info + ")"); + throw new MetaException("Unable to connect to transaction database " + + e.getMessage()); + } finally { + closeStmt(pStmt); + } + } + /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by * min(max(TXNS.txn_id), min(WRITE_SET.WS_COMMIT_ID), min(Aborted TXNS.txn_id)). @@ -1528,16 +1586,46 @@ public void markRefused(CompactionInfo info) throws MetaException { @Override @RetrySemantics.CannotRetry - public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { + public void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isAbort) throws MetaException { try { try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { - try (PreparedStatement stmt = dbConn.prepareStatement("UPDATE \"COMPACTION_QUEUE\" " + - "SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?")) { + String query; + if (isAbort) { + // Check whether we need to do an insert to the TXN_CLEANUP_QUEUE or an update to TXN_CLEANUP_QUEUE. + if (!info.hasOldAbort) { + if (info.partName != null) { + query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + + "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; + } else { + query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + + "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; + } + } else { + query = "UPDATE \"TXN_CLEANUP_QUEUE\" SET \"TCQ_RETRY_RETENTION\" = ?, \"TCQ_ERROR_MESSAGE\" = ? " + + "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND \"TCQ_PARTITION\" " + (info.partName != null ? "= ?" : "IS NULL"); + } + } else { + query = "UPDATE \"COMPACTION_QUEUE\" SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?"; + } + try (PreparedStatement stmt = dbConn.prepareStatement(query)) { stmt.setLong(1, info.retryRetention); stmt.setString(2, info.errorMessage); - stmt.setLong(3, info.id); + if (isAbort) { + stmt.setString(3, info.dbname); + stmt.setString(4, info.tableName); + if (info.partName != null) { + stmt.setString(5, info.partName); + } + } else { + stmt.setLong(3, info.id); + } int updCnt = stmt.executeUpdate(); - if (updCnt != 1) { + if (isAbort && updCnt == 0) { + LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); + dbConn.rollback(); + throw new MetaException("No record with TCQ_DATABASE=" + info.dbname + ", TCQ_TABLE=" + + info.tableName + ", TCQ_PARTITION" + info.partName + " found in TXN_CLEANUP_QUEUE"); + } else if (!isAbort && updCnt != 1) { LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); dbConn.rollback(); throw new MetaException("No record with CQ_ID=" + info.id + " found in COMPACTION_QUEUE"); @@ -1557,7 +1645,7 @@ public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws Meta throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - setCleanerRetryRetentionTimeOnError(info); + setCleanerRetryRetentionTimeOnError(info, isAbort); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index cc8f9d94a26d..360543bddf94 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -523,11 +523,12 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * or partition that will trigger cleanup. * @param abortedThreshold Number of aborted transactions involving a given table or partition * that will trigger cleanup. + * @param retentionTime Milliseconds to delay the cleaner * @return Information of potential abort items that needs to be cleaned. * @throws MetaException */ @RetrySemantics.ReadOnly - List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException; + List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold, long retentionTime) throws MetaException; /** * Sets the cleaning start time for a particular compaction @@ -577,10 +578,11 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * Stores the value of {@link CompactionInfo#retryRetention} and {@link CompactionInfo#errorMessage} fields * of the CompactionInfo in the HMS database. * @param info The {@link CompactionInfo} object holding the values. + * @param isAbort Whether the entry is associated to compaction/abort cleanup. * @throws MetaException */ @RetrySemantics.CannotRetry - void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; + void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isAbort) throws MetaException; /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql index 68f9dbe08635..03b94b4a6bae 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql @@ -640,6 +640,15 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ); +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE clob +); + CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( NCQ_NEXT bigint NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql index afd38a400319..6d99f37c9150 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql @@ -26,5 +26,15 @@ CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("DB_NAME", "TAB DROP INDEX "APP"."PCS_STATS_IDX"; CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME","CAT_NAME"); +-- HIVE-27332 +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE clob +); + -- This needs to be the last thing done. Insert any changes above this line. UPDATE "APP".VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; \ No newline at end of file diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql index df643b7432a8..05344b5ce116 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql @@ -1063,6 +1063,15 @@ PRIMARY KEY CLUSTERED ) ); +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint NULL, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE varchar(max) NULL +); + CREATE TABLE COMPLETED_COMPACTIONS ( CC_ID bigint NOT NULL, CC_DATABASE nvarchar(128) NOT NULL, diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql index 235b52ee741f..bece0e64a7a0 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql @@ -26,6 +26,16 @@ CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PAR DROP INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS; CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAME, CAT_NAME); +-- HIVE-27332 +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint NULL, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE varchar(max) NULL +); + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql index 99a12998f10f..3da816a7efeb 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql @@ -1101,6 +1101,15 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ) ENGINE=InnoDB DEFAULT CHARSET=latin1; +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE mediumtext +) ENGINE=InnoDB DEFAULT CHARSET=latin1; + CREATE TABLE COMPLETED_COMPACTIONS ( CC_ID bigint PRIMARY KEY, CC_DATABASE varchar(128) NOT NULL, diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql index 67cede7d8c6d..6191e7a43413 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql @@ -27,6 +27,16 @@ CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAM DROP INDEX PCS_STATS_IDX ON PART_COL_STATS; CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME,CAT_NAME) USING BTREE; +-- HIVE-27332 +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME bigint, + TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, + TCQ_ERROR_MESSAGE mediumtext +) ENGINE=InnoDB DEFAULT CHARSET=latin1; + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql index b9c98928046f..d75c97aa3290 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql @@ -1103,6 +1103,15 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ) ROWDEPENDENCIES; +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME NUMBER(19), + TCQ_RETRY_RETENTION NUMBER(19) DEFAULT 0 NOT NULL, + TCQ_ERROR_MESSAGE CLOB +); + CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( NCQ_NEXT NUMBER(19) NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql index c33e400e460c..c16679eabf89 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql @@ -26,6 +26,16 @@ CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAM DROP INDEX PCS_STATS_IDX; CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME,CAT_NAME); +-- HIVE-27332 +CREATE TABLE TXN_CLEANUP_QUEUE ( + TCQ_DATABASE varchar(128) NOT NULL, + TCQ_TABLE varchar(256) NOT NULL, + TCQ_PARTITION varchar(767), + TCQ_RETRY_TIME NUMBER(19), + TCQ_RETRY_RETENTION NUMBER(19) DEFAULT 0 NOT NULL, + TCQ_ERROR_MESSAGE CLOB +); + -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS Status from dual; diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql index 032989a3302b..26d1f7f76e45 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql @@ -1818,6 +1818,15 @@ CREATE TABLE "COMPACTION_QUEUE" ( "CQ_ORDER_BY" varchar(4000) ); +CREATE TABLE "TXN_CLEANUP_QUEUE" ( + "TCQ_DATABASE" varchar(128) NOT NULL, + "TCQ_TABLE" varchar(256) NOT NULL, + "TCQ_PARTITION" varchar(767), + "TCQ_RETRY_TIME" bigint, + "TCQ_RETRY_RETENTION" bigint NOT NULL DEFAULT 0, + "TCQ_ERROR_MESSAGE" text +); + CREATE TABLE "NEXT_COMPACTION_QUEUE_ID" ( "NCQ_NEXT" bigint NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql index 041d9f55032a..8a9ed46565cc 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql @@ -26,6 +26,16 @@ CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("DB_NAME","TABL DROP INDEX "PCS_STATS_IDX"; CREATE INDEX "PCS_STATS_IDX" ON "PART_COL_STATS" USING btree ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME","CAT_NAME"); +-- HIVE-27332 +CREATE TABLE "TXN_CLEANUP_QUEUE" ( + "TCQ_DATABASE" varchar(128) NOT NULL, + "TCQ_TABLE" varchar(256) NOT NULL, + "TCQ_PARTITION" varchar(767), + "TCQ_RETRY_TIME" bigint, + "TCQ_RETRY_RETENTION" bigint NOT NULL DEFAULT 0, + "TCQ_ERROR_MESSAGE" text +); + -- These lines need to be last. Insert any changes above. UPDATE "VERSION" SET "SCHEMA_VERSION"='4.0.0', "VERSION_COMMENT"='Hive release version 4.0.0' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0'; diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java index 9e712a9c54b6..6a2a3c9a0bf9 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java @@ -223,6 +223,7 @@ public static void cleanDb(Configuration conf) throws Exception { success &= truncateTable(conn, conf, stmt, "HIVE_LOCKS"); success &= truncateTable(conn, conf, stmt, "NEXT_LOCK_ID"); success &= truncateTable(conn, conf, stmt, "COMPACTION_QUEUE"); + success &= truncateTable(conn, conf, stmt, "TXN_CLEANUP_QUEUE"); success &= truncateTable(conn, conf, stmt, "NEXT_COMPACTION_QUEUE_ID"); success &= truncateTable(conn, conf, stmt, "COMPLETED_COMPACTIONS"); success &= truncateTable(conn, conf, stmt, "AUX_TABLE"); From a0492ab88b86eccffac11d3b60ff0f522fa0052c Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Wed, 17 May 2023 19:53:51 +0530 Subject: [PATCH 2/6] Address review comments - 1 --- .../hive/ql/txn/compactor/CompactorTest.java | 2 +- .../TestAbortCleanupUsingCompactionCycle.java | 4 ++-- ...gCompactionCycleWithMinHistoryWriteId.java | 4 ++-- .../hive/ql/txn/compactor/TestCleaner.java | 2 -- .../metastore/txn/CompactionTxnHandler.java | 22 +++++++++++-------- 5 files changed, 18 insertions(+), 16 deletions(-) diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java index 1899c989a15c..ba90d8549d19 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/CompactorTest.java @@ -128,7 +128,7 @@ public abstract class CompactorTest { private final AtomicBoolean stop = new AtomicBoolean(); private Path tmpdir; FileSystem fs; - + @Before @BeforeEach public void setup() throws Exception { diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycle.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycle.java index b57599113e6e..01625d431129 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycle.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycle.java @@ -18,11 +18,11 @@ package org.apache.hadoop.hive.ql.txn.compactor; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestAbortCleanupUsingCompactionCycle extends TestCleaner { @Override - @Before + @BeforeEach public void setup() throws Exception { super.setup(); MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, false); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycleWithMinHistoryWriteId.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycleWithMinHistoryWriteId.java index bbc72661abf1..77782534ec42 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycleWithMinHistoryWriteId.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestAbortCleanupUsingCompactionCycleWithMinHistoryWriteId.java @@ -18,11 +18,11 @@ package org.apache.hadoop.hive.ql.txn.compactor; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.junit.Before; +import org.junit.jupiter.api.BeforeEach; public class TestAbortCleanupUsingCompactionCycleWithMinHistoryWriteId extends TestCleaner { @Override - @Before + @BeforeEach public void setup() throws Exception { super.setup(); MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, false); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java index 01c9f09803f0..201b4ee5749f 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java @@ -44,7 +44,6 @@ import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandler; import org.apache.hadoop.hive.ql.txn.compactor.handler.TaskHandlerFactory; -import org.junit.After; import org.junit.Assert; import org.junit.Test; import org.junit.jupiter.api.AfterEach; @@ -811,7 +810,6 @@ boolean useHive130DeltaDirName() { return false; } - @After @AfterEach public void tearDown() throws Exception { compactorTestCleanup(); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 750bb04a01d2..b899b145fe3f 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -40,6 +40,7 @@ import java.sql.ResultSet; import java.sql.SQLException; import java.sql.Statement; +import java.sql.Types; import java.util.ArrayList; import java.util.Collections; import java.util.HashSet; @@ -808,7 +809,7 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws private void removeRetryQueueEntries(Connection dbConn, CompactionInfo info) throws MetaException, RetryException { PreparedStatement pStmt = null; String query = "DELETE FROM \"TXN_CLEANUP_QUEUE\" WHERE \"TCQ_DATABASE\" = ? " + - "AND \"TCQ_TABLE\" = ? AND \"TCQ_PARTITION\" " + (info.partName != null ? "= ?" : "IS NULL"); + "AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; try { LOG.debug("Going to execute update <{}>", query); pStmt = dbConn.prepareStatement(query); @@ -816,6 +817,10 @@ private void removeRetryQueueEntries(Connection dbConn, CompactionInfo info) thr pStmt.setString(2, info.tableName); if (info.partName != null) { pStmt.setString(3, info.partName); + } else { + // Since the type of 'TCQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + pStmt.setNull(3, Types.VARCHAR); } int rc = pStmt.executeUpdate(); LOG.debug("Removed {} records in txn_cleanup_queue", rc); @@ -1593,16 +1598,11 @@ public void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isA if (isAbort) { // Check whether we need to do an insert to the TXN_CLEANUP_QUEUE or an update to TXN_CLEANUP_QUEUE. if (!info.hasOldAbort) { - if (info.partName != null) { - query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + - "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; - } else { - query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + - "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; - } + query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + + "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; } else { query = "UPDATE \"TXN_CLEANUP_QUEUE\" SET \"TCQ_RETRY_RETENTION\" = ?, \"TCQ_ERROR_MESSAGE\" = ? " + - "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND \"TCQ_PARTITION\" " + (info.partName != null ? "= ?" : "IS NULL"); + "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; } } else { query = "UPDATE \"COMPACTION_QUEUE\" SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?"; @@ -1615,6 +1615,10 @@ public void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isA stmt.setString(4, info.tableName); if (info.partName != null) { stmt.setString(5, info.partName); + } else { + // Since the type of 'TCQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + stmt.setNull(5, Types.VARCHAR); } } else { stmt.setLong(3, info.id); From e932651e8eb6e9d9914fa3b6f37964e8eccb494d Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Tue, 23 May 2023 12:20:00 +0530 Subject: [PATCH 3/6] Address review comments - 2 --- .../compactor/handler/AbortedTxnCleaner.java | 37 +-- .../compactor/handler/CompactionCleaner.java | 22 +- .../ql/txn/compactor/handler/TaskHandler.java | 21 -- .../handler/TestAbortedTxnCleaner.java | 166 +++++++++--- .../metastore/txn/AbortTxnRequestInfo.java | 41 +++ .../metastore/txn/CompactionTxnHandler.java | 251 +++++++++++------- .../hadoop/hive/metastore/txn/TxnStore.java | 16 +- 7 files changed, 372 insertions(+), 182 deletions(-) create mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index 26039ca4ca9d..7bfe6cd6b2a5 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; -import org.apache.hadoop.hive.metastore.txn.CompactionInfo; +import org.apache.hadoop.hive.metastore.txn.AbortTxnRequestInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; @@ -41,8 +41,8 @@ import java.util.stream.Collectors; import static java.util.Objects.isNull; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME; -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; /** * Abort-cleanup based implementation of TaskHandler. @@ -65,7 +65,7 @@ aborted txns are not visible by any open txns. It is only visible while determin only sees the aborted deltas and does not read the file).

The following algorithm is used to clean the set of aborted directories -
- a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanAborts(long, int, long)}).
+ a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanAborts(long, int)}).
b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table.
c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID
d. Fetch the aborted directories and delete the directories.
@@ -78,20 +78,17 @@ public List getTasks() throws MetaException { long abortedTimeThreshold = HiveConf .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, TimeUnit.MILLISECONDS); - long retentionTime = HiveConf.getBoolVar(conf, HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED) - ? HiveConf.getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETENTION_TIME, TimeUnit.MILLISECONDS) - : 0; - List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold, retentionTime); + List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); if (!readyToCleanAborts.isEmpty()) { - return readyToCleanAborts.stream().map(ci -> ThrowingRunnable.unchecked(() -> - clean(ci, ci.txnId > 0 ? ci.txnId : Long.MAX_VALUE, metricsEnabled))) + return readyToCleanAborts.stream().map(info -> ThrowingRunnable.unchecked(() -> + clean(info, info.minOpenWriteTxnId > 0 ? info.minOpenWriteTxnId : Long.MAX_VALUE, metricsEnabled))) .collect(Collectors.toList()); } return Collections.emptyList(); } - private void clean(CompactionInfo info, long minOpenWriteTxn, boolean metricsEnabled) throws MetaException, InterruptedException { + private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metricsEnabled) throws MetaException, InterruptedException { LOG.info("Starting cleaning for {}", info); PerfLogger perfLogger = PerfLogger.getPerfLogger(false); String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_"; @@ -126,13 +123,13 @@ private void clean(CompactionInfo info, long minOpenWriteTxn, boolean metricsEna LOG.error("Caught an interrupted exception when cleaning, unable to complete cleaning of {} due to {}", info, e.getMessage()); info.errorMessage = e.getMessage(); - handleCleanerAttemptFailure(info, true); + handleCleanerAttemptFailure(info); throw e; } catch (Exception e) { LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info, e.getMessage()); info.errorMessage = e.getMessage(); - handleCleanerAttemptFailure(info, true); + handleCleanerAttemptFailure(info); throw new MetaException(e.getMessage()); } finally { if (metricsEnabled) { @@ -141,7 +138,7 @@ private void clean(CompactionInfo info, long minOpenWriteTxn, boolean metricsEna } } - private void abortCleanUsingAcidDir(CompactionInfo info, String location, long minOpenWriteTxn) throws Exception { + private void abortCleanUsingAcidDir(AbortTxnRequestInfo info, String location, long minOpenWriteTxn) throws Exception { ValidTxnList validTxnList = TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenWriteTxn, true); //save it so that getAcidState() sees it @@ -160,10 +157,20 @@ private void abortCleanUsingAcidDir(CompactionInfo info, String location, long m boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table); if (success || CompactorUtil.isDynPartAbort(table, info.partName)) { - txnHandler.markCleaned(info, false); + txnHandler.markCleaned(info, true); } else { LOG.warn("Leaving aborted entry {} in TXN_COMPONENTS table.", info); } } + + private void handleCleanerAttemptFailure(AbortTxnRequestInfo info) throws MetaException { + long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); + int cleanAttempts = 0; + if (info.retryRetention > 0) { + cleanAttempts = (int)(Math.log(info.retryRetention / defaultRetention) / Math.log(2)) + 1; + } + info.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; + txnHandler.setAbortCleanerRetryRetentionTimeOnError(info); + } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java index 7920a53788a7..f7c8944ea3ee 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java @@ -58,6 +58,10 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; import static java.util.Objects.isNull; /** @@ -168,7 +172,7 @@ private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) t if (metricsEnabled) { Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc(); } - handleCleanerAttemptFailure(ci, false); + handleCleanerAttemptFailure(ci); } finally { if (metricsEnabled) { perfLogger.perfLogEnd(CompactionCleaner.class.getName(), cleanerMetric); @@ -298,6 +302,22 @@ protected ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, V return validWriteIdList; } + private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException { + long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); + int cleanAttempts = 0; + if (ci.retryRetention > 0) { + cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1; + } + if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) { + //Mark it as failed if the max attempt threshold is reached. + txnHandler.markFailed(ci); + } else { + //Calculate retry retention time and update record. + ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; + txnHandler.setCleanerRetryRetentionTimeOnError(ci); + } + } + private CleanupRequest getCleaningRequestBasedOnLocation(CompactionInfo ci, String location) { String strIfPurge = ci.getProperty("ifPurge"); boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge")); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index 3674c47910d1..ef95a100c1a2 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -49,13 +49,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import static org.apache.commons.collections.ListUtils.subtract; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; /** * An abstract class which defines the list of utility methods for performing cleanup activities. @@ -166,20 +161,4 @@ protected boolean cleanAndVerifyObsoleteDirectories(CompactionInfo info, String return success; } - - protected void handleCleanerAttemptFailure(CompactionInfo ci, boolean isAbort) throws MetaException { - long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); - int cleanAttempts = 0; - if (ci.retryRetention > 0) { - cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1; - } - if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS) && !isAbort) { - //Mark it as failed if the max attempt threshold is reached. - txnHandler.markFailed(ci); - } else { - //Calculate retry retention time and update record. - ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; - txnHandler.setCleanerRetryRetentionTimeOnError(ci, isAbort); - } - } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 9eba02eafd16..8acaadfa4b04 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -26,6 +26,7 @@ import org.apache.hadoop.hive.metastore.api.Partition; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; +import org.apache.hadoop.hive.metastore.txn.AbortTxnRequestInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -46,8 +47,8 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.hadoop.hive.ql.TxnCommandsBaseForTests.runInitiator; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; public class TestAbortedTxnCleaner extends TestHandler { @@ -329,6 +330,54 @@ public void testAbortedCleaningWithThreeTxnsWithDiffWriteIds() throws Exception Assert.assertEquals(5, directories.size()); } + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testAbortCleanupNotUpdatingCompactionTables(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "abort_cleanup_not_populating_compaction_tables_test", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // 3-aborted deltas & one committed delta + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + + MetastoreConf.setBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER, true); + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler mockedTaskHandler = Mockito.spy(new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover)); + + runInitiator(conf); + String compactionQueuePresence = "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + + " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, compactionQueuePresence)); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); + cleaner.run(); + + Mockito.verify(mockedFSRemover, Mockito.times(1)).clean(any(CleanupRequest.class)); + Mockito.verify(mockedTaskHandler, Mockito.times(1)).getTasks(); + + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, compactionQueuePresence)); + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPLETED_COMPACTIONS\" " + + " WHERE \"CC_DATABASE\" = '" + dbName+ "' AND \"CC_TABLE\" = '" + tableName + "' AND \"CC_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"))); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPLETED_TXN_COMPONENTS\" " + + " WHERE \"CTC_DATABASE\" = '" + dbName+ "' AND \"CTC_TABLE\" = '" + tableName + "' AND \"CTC_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"))); + + List directories = getDirectories(conf, t, null); + // All aborted directories removed, hence 1 committed delta directory must be present + Assert.assertEquals(1, directories.size()); + } + @ParameterizedTest @ValueSource(booleans = {true, false}) public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { @@ -359,18 +408,16 @@ public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); cleaner.run(); - Mockito.verify(mockedTxnHandler, Mockito.times(1)).setCleanerRetryRetentionTimeOnError(any(), eq(true)); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + Mockito.verify(mockedTxnHandler, Mockito.times(1)).setAbortCleanerRetryRetentionTimeOnError(any(AbortTxnRequestInfo.class)); + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); } @@ -408,16 +455,14 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -432,12 +477,12 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); } @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testRetryWithNoDelay(boolean isPartitioned) throws Exception { + public void testRetryWithinRetentionTime(boolean isPartitioned) throws Exception { String dbName = "default", tableName = "handler_retry_nodelay", partName = "today"; Table t = newTable(dbName, tableName, isPartitioned); Partition p = isPartitioned ? newPartition(t, partName) : null; @@ -464,17 +509,15 @@ public void testRetryWithNoDelay(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -487,7 +530,7 @@ public void testRetryWithNoDelay(boolean isPartitioned) throws Exception { cleaner.run(); // The retry entry is not removed since retry conditions are not achieved hence its not picked for cleanup. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -529,16 +572,14 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -552,7 +593,7 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.run(); // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -594,16 +635,14 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + - "WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery) .replace("\n", "").trim().contains("Testing first retry")); @@ -621,7 +660,7 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.run(); // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\"")); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -629,4 +668,57 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing second retry")); } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testZeroRetryRetentionTimeForAbortCleanup(boolean isPartitioned) throws Exception { + String dbName = "default", tableName = "handler_zero_retryretention", partName = "today"; + Table t = newTable(dbName, tableName, isPartitioned); + Partition p = isPartitioned ? newPartition(t, partName) : null; + + // Add 2 committed deltas and 2 aborted deltas + addDeltaFileWithTxnComponents(t, p, 2, false); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, true); + addDeltaFileWithTxnComponents(t, p, 2, false); + + HiveConf.setIntVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_THRESHOLD, 0); + MetastoreConf.setTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, 0, TimeUnit.MILLISECONDS); + MetadataCache metadataCache = new MetadataCache(true); + FSRemover mockedFSRemover = Mockito.spy(new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache)); + TaskHandler taskHandler = new AbortedTxnCleaner(conf, txnHandler, metadataCache, + false, mockedFSRemover); + // Invoke runtime exception when calling markCleaned. + Mockito.doAnswer(invocationOnMock -> { + throw new RuntimeException("Testing retry"); + }).when(mockedFSRemover).clean(any()); + + Cleaner cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + Assert.assertEquals(Integer.toString(0), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) + .replace("\n", "").trim()); + // Cast clob to varchar to get the string output + String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) + .replace("\n", "").trim().contains("Testing retry")); + + Mockito.doAnswer(InvocationOnMock::callRealMethod).when(mockedFSRemover).clean(any()); + + cleaner = new Cleaner(); + cleaner.setConf(conf); + cleaner.init(new AtomicBoolean(true)); + cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); + cleaner.run(); + + // The retry entry should be removed since retry conditions are achieved because retry retention time is 0. + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java new file mode 100644 index 000000000000..356ed88a19ee --- /dev/null +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java @@ -0,0 +1,41 @@ +/* + * 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.metastore.txn; + +import org.apache.commons.lang3.builder.ToStringBuilder; + +public class AbortTxnRequestInfo extends CompactionInfo { + public boolean hasRetryEntry; + public long minOpenWriteTxnId = 0; + + public AbortTxnRequestInfo() {} + + public String toString() { + return new ToStringBuilder(this) + .append("dbname", dbname) + .append("tableName", tableName) + .append("partName", partName) + .append("runAs", runAs) + .append("minOpenWriteTxnId", minOpenWriteTxnId) + .append("highestWriteId", highestWriteId) + .append("errorMessage", errorMessage) + .append("retryRetention", retryRetention) + .append("hasRetryEntry", hasRetryEntry) + .build(); + } +} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index b899b145fe3f..0cee8f121c6c 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -93,6 +93,39 @@ class CompactionTxnHandler extends TxnHandler { "DELETE FROM \"TXNS\" WHERE \"TXN_ID\" NOT IN (SELECT \"TC_TXNID\" FROM \"TXN_COMPONENTS\") " + "AND (\"TXN_STATE\" = " + TxnStatus.ABORTED + " OR \"TXN_STATE\" = " + TxnStatus.COMMITTED + ") " + "AND \"TXN_ID\" < ?"; + + // Checks presence of the retry entry (by assigning 0 in retry retention) for the corresponding table/partition in the TXN_CLEANUP_QUEUE + private static final String SELECT_TXN_CLEANUP_QUEUE_PRESENCE_QUERY = + " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", " + + "-1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + + "-1 AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + "GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\""; + + private static final String SELECT_RETRY_RETENTION_QUERY = + " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", " + + " -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + + " MAX(\"TCQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + + " GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\" " + + "HAVING MIN(\"TCQ_RETRY_TIME\") < %s - MAX(\"TCQ_RETRY_RETENTION\")"; + + private static final String SELECT_MIN_OPEN_WRITETXN_ID_QUERY = + " SELECT MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + + " FROM \"TXNS\", \"TXN_COMPONENTS\" WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\"=" + TxnStatus.OPEN + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\""; + + private static final String SELECT_ABORTED_TXNS_QUERY = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " + + " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" "; + + private static final String SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY = "SELECT \"tc\".\"TC_DATABASE\" AS \"DB\", \"tc\".\"TC_TABLE\" AS \"TBL\", " + + " \"tc\".\"TC_PARTITION\" AS \"PART\", \"tc\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", " + + " \"tc\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", \"minOpenWriteTxnId\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", -2 AS \"RETRY_RETENTION\" " + + " FROM ( " + SELECT_ABORTED_TXNS_QUERY + " %s ) \"tc\" LEFT JOIN ( " + SELECT_MIN_OPEN_WRITETXN_ID_QUERY + " ) \"minOpenWriteTxnId\" " + + " ON \"tc\".\"TC_DATABASE\" = \"minOpenWriteTxnId\".\"TC_DATABASE\" AND \"tc\".\"TC_TABLE\" = \"minOpenWriteTxnId\".\"TC_TABLE\"" + + " AND (\"tc\".\"TC_PARTITION\" = \"minOpenWriteTxnId\".\"TC_PARTITION\" OR (\"tc\".\"TC_PARTITION\" IS NULL AND \"minOpenWriteTxnId\".\"TC_PARTITION\" IS NULL))"; + + public CompactionTxnHandler() { } @@ -473,38 +506,21 @@ public List findReadyToClean(long minOpenTxnWaterMark, long rete @Override @RetrySemantics.ReadOnly - public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold, long retentionTime) throws MetaException { + public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException { try { - List readyToCleanAborts = new ArrayList<>(); + List readyToCleanAborts = new ArrayList<>(); try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction); Statement stmt = dbConn.createStatement()) { boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0; - String firstInnerQuery = "SELECT \"tc\".\"TC_DATABASE\" AS \"DB\", \"tc\".\"TC_TABLE\" AS \"TBL\", \"tc\".\"TC_PARTITION\" AS \"PART\", " + - " \"tc\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", \"tc\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", " + - " \"minOpenWriteTxnId\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", -1 AS \"RETRY_RETENTION\" FROM " + - " ( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " + - " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + - " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + - (checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold) + " ) \"tc\" " + - " LEFT JOIN ( SELECT MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + - " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\"=" + TxnStatus.OPEN + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" ) \"minOpenWriteTxnId\" " + - " ON \"tc\".\"TC_DATABASE\" = \"minOpenWriteTxnId\".\"TC_DATABASE\" AND \"tc\".\"TC_TABLE\" = \"minOpenWriteTxnId\".\"TC_TABLE\"" + - " AND (\"tc\".\"TC_PARTITION\" = \"minOpenWriteTxnId\".\"TC_PARTITION\" OR (\"tc\".\"TC_PARTITION\" IS NULL AND \"minOpenWriteTxnId\".\"TC_PARTITION\" IS NULL))"; - - String secondInnerQuery = " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", " + - " 0 AS \"MIN_OPEN_WRITE_TXNID\", MAX(\"TCQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - " GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\" " + - " HAVING MIN(\"TCQ_RETRY_TIME\") < " + getEpochFn(dbProduct) + " - MAX(\"TCQ_RETRY_RETENTION\") - " + retentionTime; - - // Checks presence of the retry entry (by assigning 0 in retry retention) for the corresponding table/partition in the TXN_CLEANUP_QUEUE - String thirdInnerQuery = " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", " + - "0 AS \"MIN_OPEN_WRITE_TXNID\", 0 AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\""; - - String unionQuery = firstInnerQuery + " UNION ALL " + secondInnerQuery + " UNION ALL " + thirdInnerQuery; - - String sCheckAborted = "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\") " + - "FROM ( " + unionQuery + " ) res GROUP BY \"DB\", \"TBL\", \"PART\""; + String firstInnerQuery = String.format(SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY, + checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold); + + String secondInnerQuery = String.format(SELECT_RETRY_RETENTION_QUERY, getEpochFn(dbProduct)); + + String unionQuery = firstInnerQuery + " UNION ALL " + secondInnerQuery + " UNION ALL " + SELECT_TXN_CLEANUP_QUEUE_PRESENCE_QUERY; + + String sCheckAborted = "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), " + + "MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\") FROM ( " + unionQuery + " ) res GROUP BY \"DB\", \"TBL\", \"PART\""; LOG.debug("Going to execute query <{}>", sCheckAborted); try (ResultSet rs = stmt.executeQuery(sCheckAborted)) { @@ -514,21 +530,21 @@ public List findReadyToCleanAborts(long abortedTimeThreshold, in checkAbortedTimeThreshold && rs.getLong(4) + abortedTimeThreshold < systemTime; int numAbortedTxns = rs.getInt(5); if (numAbortedTxns > abortedThreshold || pastTimeThreshold) { - CompactionInfo info = new CompactionInfo(); + AbortTxnRequestInfo info = new AbortTxnRequestInfo(); info.dbname = rs.getString(1); info.tableName = rs.getString(2); info.partName = rs.getString(3); // In this case, this field contains min open write txn ID. - info.txnId = rs.getLong(6); + info.minOpenWriteTxnId = rs.getLong(6); // The query returns the following retry retention values - 3 cases exist - - // -1 - No retry entry in TXN_CLEANUP_QUEUE - // 0 - Retry entry is present but hasn't achieved conditions for retry (i.e. min(tcq_retry_time) >= current_timestamp - max(retry_retention_time) - retentionTime) - // > 0 - Retry entry is present and has achieved conditions for retry (i.e. min(tcq_retry_time) < current_timestamp - max(retry_retention_time) - retentionTime) + // -2 - No retry entry in TXN_CLEANUP_QUEUE + // -1 - Retry entry is present but hasn't achieved conditions for retry (i.e. min(tcq_retry_time) >= current_timestamp - max(retry_retention_time)) + // >= 0 - Retry entry is present and has achieved conditions for retry (i.e. min(tcq_retry_time) < current_timestamp - max(retry_retention_time)) info.retryRetention = rs.getLong(7); - // Do not add entry when retry retention is 0. - if (info.retryRetention != 0) { + // Do not add entry when retry retention is -1. + if (info.retryRetention != -1) { // In this case, this flag is used for checking whether a retry entry needs to be inserted or updated. - info.hasOldAbort = info.retryRetention > 0; + info.hasRetryEntry = info.retryRetention >= 0; readyToCleanAborts.add(info); } } @@ -541,7 +557,7 @@ public List findReadyToCleanAborts(long abortedTimeThreshold, in throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - return findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold, retentionTime); + return findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); } } @@ -739,7 +755,7 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws * or abort cleanup is successful. We dont want a situation wherein we have a retry queue entry for a table * but no corresponding entry in TXN_COMPONENTS table. */ - removeRetryQueueEntries(dbConn, info); + TxnCleanupQueueHandler.removeRetryQueueEntries(this, dbConn, info); /* * compaction may remove data from aborted txns above tc_writeid bit it only guarantees to @@ -806,36 +822,6 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws } } - private void removeRetryQueueEntries(Connection dbConn, CompactionInfo info) throws MetaException, RetryException { - PreparedStatement pStmt = null; - String query = "DELETE FROM \"TXN_CLEANUP_QUEUE\" WHERE \"TCQ_DATABASE\" = ? " + - "AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; - try { - LOG.debug("Going to execute update <{}>", query); - pStmt = dbConn.prepareStatement(query); - pStmt.setString(1, info.dbname); - pStmt.setString(2, info.tableName); - if (info.partName != null) { - pStmt.setString(3, info.partName); - } else { - // Since the type of 'TCQ_PARTITION' column is varchar. - // Hence, setting null for VARCHAR type. - pStmt.setNull(3, Types.VARCHAR); - } - int rc = pStmt.executeUpdate(); - LOG.debug("Removed {} records in txn_cleanup_queue", rc); - } catch (SQLException e) { - LOG.error("Unable to delete from txn components due to {}", e.getMessage()); - LOG.debug("Going to rollback"); - rollbackDBConn(dbConn); - checkRetryable(e, "removeRetryQueueEntries(" + info + ")"); - throw new MetaException("Unable to connect to transaction database " + - e.getMessage()); - } finally { - closeStmt(pStmt); - } - } - /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by * min(max(TXNS.txn_id), min(WRITE_SET.WS_COMMIT_ID), min(Aborted TXNS.txn_id)). @@ -1266,6 +1252,79 @@ private static class RetentionCounters { } } + static class TxnCleanupQueueHandler { + + public static void removeRetryQueueEntries(CompactionTxnHandler txnHandler, + Connection dbConn, CompactionInfo info) throws MetaException, RetryException { + PreparedStatement pStmt = null; + String query = "DELETE FROM \"TXN_CLEANUP_QUEUE\" WHERE \"TCQ_DATABASE\" = ? " + + "AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; + try { + LOG.debug("Going to execute update <{}>", query); + pStmt = dbConn.prepareStatement(query); + pStmt.setString(1, info.dbname); + pStmt.setString(2, info.tableName); + if (info.partName != null) { + pStmt.setString(3, info.partName); + } else { + // Since the type of 'TCQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + pStmt.setNull(3, Types.VARCHAR); + } + int rc = pStmt.executeUpdate(); + LOG.debug("Removed {} records in txn_cleanup_queue", rc); + } catch (SQLException e) { + LOG.error("Unable to delete from txn components due to {}", e.getMessage()); + LOG.debug("Going to rollback"); + rollbackDBConn(dbConn); + txnHandler.checkRetryable(e, "removeRetryQueueEntries(" + info + ")"); + throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); + } finally { + closeStmt(pStmt); + } + } + + public static void setRetryRetentionTimeOnError(CompactionTxnHandler txnHandler, + Connection dbConn, AbortTxnRequestInfo info) throws MetaException, RetryException { + String query; + if (!info.hasRetryEntry) { + query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + + "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; + } else { + query = "UPDATE \"TXN_CLEANUP_QUEUE\" SET \"TCQ_RETRY_RETENTION\" = ?, \"TCQ_ERROR_MESSAGE\" = ? " + + "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; + } + try (PreparedStatement stmt = dbConn.prepareStatement(query)) { + stmt.setLong(1, info.retryRetention); + stmt.setString(2, info.errorMessage); + stmt.setString(3, info.dbname); + stmt.setString(4, info.tableName); + if (info.partName != null) { + stmt.setString(5, info.partName); + } else { + // Since the type of 'TCQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + stmt.setNull(5, Types.VARCHAR); + } + int updCnt = stmt.executeUpdate(); + if (updCnt == 0) { + LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); + dbConn.rollback(); + throw new MetaException("No record with TCQ_DATABASE=" + info.dbname + ", TCQ_TABLE=" + + info.tableName + ", TCQ_PARTITION" + info.partName + " found in TXN_CLEANUP_QUEUE"); + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + LOG.error("Unable to update compaction queue: " + e.getMessage()); + rollbackDBConn(dbConn); + txnHandler.checkRetryable(e, "setCleanerRetryRetentionTimeOnError(" + info + ")"); + throw new MetaException("Unable to update compaction queue: " + + e.getMessage()); + } + } + } + private void checkForDeletion(List deleteSet, CompactionInfo ci, RetentionCounters rc, long timeoutThreshold) { switch (ci.state) { case DID_NOT_INITIATE: @@ -1588,48 +1647,34 @@ public void markRefused(CompactionInfo info) throws MetaException { updateStatus(info); } + @Override + @RetrySemantics.CannotRetry + public void setAbortCleanerRetryRetentionTimeOnError(AbortTxnRequestInfo info) throws MetaException { + try { + try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { + TxnCleanupQueueHandler.setRetryRetentionTimeOnError(this, dbConn, info); + } catch (SQLException e) { + LOG.error(DB_FAILED_TO_CONNECT + e.getMessage()); + checkRetryable(e, "setAbortCleanerRetryRetentionTimeOnError(" + info + ")"); + throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); + } + } catch (RetryException e) { + setAbortCleanerRetryRetentionTimeOnError(info); + } + } @Override @RetrySemantics.CannotRetry - public void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isAbort) throws MetaException { + public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { try { try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { - String query; - if (isAbort) { - // Check whether we need to do an insert to the TXN_CLEANUP_QUEUE or an update to TXN_CLEANUP_QUEUE. - if (!info.hasOldAbort) { - query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + - "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; - } else { - query = "UPDATE \"TXN_CLEANUP_QUEUE\" SET \"TCQ_RETRY_RETENTION\" = ?, \"TCQ_ERROR_MESSAGE\" = ? " + - "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; - } - } else { - query = "UPDATE \"COMPACTION_QUEUE\" SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?"; - } - try (PreparedStatement stmt = dbConn.prepareStatement(query)) { + try (PreparedStatement stmt = dbConn.prepareStatement("UPDATE \"COMPACTION_QUEUE\" " + + "SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?")) { stmt.setLong(1, info.retryRetention); stmt.setString(2, info.errorMessage); - if (isAbort) { - stmt.setString(3, info.dbname); - stmt.setString(4, info.tableName); - if (info.partName != null) { - stmt.setString(5, info.partName); - } else { - // Since the type of 'TCQ_PARTITION' column is varchar. - // Hence, setting null for VARCHAR type. - stmt.setNull(5, Types.VARCHAR); - } - } else { - stmt.setLong(3, info.id); - } + stmt.setLong(3, info.id); int updCnt = stmt.executeUpdate(); - if (isAbort && updCnt == 0) { - LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); - dbConn.rollback(); - throw new MetaException("No record with TCQ_DATABASE=" + info.dbname + ", TCQ_TABLE=" - + info.tableName + ", TCQ_PARTITION" + info.partName + " found in TXN_CLEANUP_QUEUE"); - } else if (!isAbort && updCnt != 1) { + if (updCnt != 1) { LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); dbConn.rollback(); throw new MetaException("No record with CQ_ID=" + info.id + " found in COMPACTION_QUEUE"); @@ -1649,7 +1694,7 @@ public void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isA throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - setCleanerRetryRetentionTimeOnError(info, isAbort); + setCleanerRetryRetentionTimeOnError(info); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index 360543bddf94..2bb55ee12fc7 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -29,7 +29,6 @@ import org.apache.hadoop.hive.metastore.api.AbortTxnsRequest; import org.apache.hadoop.hive.metastore.api.AbortCompactResponse; import org.apache.hadoop.hive.metastore.api.AbortCompactionRequest; -import org.apache.hadoop.hive.metastore.api.CompactionAbortedException; import org.apache.hadoop.hive.metastore.api.AddDynamicPartitions; import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsRequest; import org.apache.hadoop.hive.metastore.api.AllocateTableWriteIdsResponse; @@ -523,12 +522,11 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * or partition that will trigger cleanup. * @param abortedThreshold Number of aborted transactions involving a given table or partition * that will trigger cleanup. - * @param retentionTime Milliseconds to delay the cleaner * @return Information of potential abort items that needs to be cleaned. * @throws MetaException */ @RetrySemantics.ReadOnly - List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold, long retentionTime) throws MetaException; + List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException; /** * Sets the cleaning start time for a particular compaction @@ -578,11 +576,19 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * Stores the value of {@link CompactionInfo#retryRetention} and {@link CompactionInfo#errorMessage} fields * of the CompactionInfo in the HMS database. * @param info The {@link CompactionInfo} object holding the values. - * @param isAbort Whether the entry is associated to compaction/abort cleanup. * @throws MetaException */ @RetrySemantics.CannotRetry - void setCleanerRetryRetentionTimeOnError(CompactionInfo info, boolean isAbort) throws MetaException; + void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; + + /** + * Stores the value of {@link AbortTxnRequestInfo#retryRetention} and {@link AbortTxnRequestInfo#errorMessage} fields + * of the AbortTxnRequestInfo in the HMS database (specifically in TXN_CLEANUP_QUEUE table). + * @param info + * @throws MetaException + */ + @RetrySemantics.CannotRetry + void setAbortCleanerRetryRetentionTimeOnError(AbortTxnRequestInfo info) throws MetaException; /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by From 3e001690210445c20b27b360a2399ef2323d4c65 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Thu, 1 Jun 2023 12:34:07 +0530 Subject: [PATCH 4/6] Address review comments - 3, Reuse compaction queue, Optimize SQL queries --- .../compactor/handler/AbortedTxnCleaner.java | 2 +- .../compactor/handler/TaskHandlerFactory.java | 4 +- .../handler/TestAbortedTxnCleaner.java | 73 ++--- .../thrift/gen-cpp/hive_metastore_types.cpp | 8 +- .../gen/thrift/gen-cpp/hive_metastore_types.h | 3 +- .../hive/metastore/api/CompactionType.java | 5 +- .../gen-php/metastore/CompactionType.php | 3 + .../thrift/gen-py/hive_metastore/ttypes.py | 3 + .../gen/thrift/gen-rb/hive_metastore_types.rb | 5 +- .../src/main/thrift/hive_metastore.thrift | 1 + .../metastore/txn/CompactionTxnHandler.java | 255 +++++++++--------- .../hadoop/hive/metastore/txn/TxnStore.java | 1 + .../hadoop/hive/metastore/txn/TxnUtils.java | 4 + .../sql/derby/hive-schema-4.0.0.derby.sql | 9 - .../upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql | 10 - .../sql/mssql/hive-schema-4.0.0.mssql.sql | 9 - .../upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql | 10 - .../sql/mysql/hive-schema-4.0.0.mysql.sql | 9 - .../upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql | 10 - .../sql/oracle/hive-schema-4.0.0.oracle.sql | 9 - .../upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql | 10 - .../postgres/hive-schema-4.0.0.postgres.sql | 9 - ...pgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql | 10 - .../hive/metastore/utils/TestTxnDbUtil.java | 1 - 24 files changed, 195 insertions(+), 268 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index 7bfe6cd6b2a5..510ee0f3aaae 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -82,7 +82,7 @@ public List getTasks() throws MetaException { if (!readyToCleanAborts.isEmpty()) { return readyToCleanAborts.stream().map(info -> ThrowingRunnable.unchecked(() -> - clean(info, info.minOpenWriteTxnId > 0 ? info.minOpenWriteTxnId : Long.MAX_VALUE, metricsEnabled))) + clean(info, info.minOpenWriteTxnId, metricsEnabled))) .collect(Collectors.toList()); } return Collections.emptyList(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java index 57a4dc625c83..33c782ce4ab8 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandlerFactory.java @@ -46,12 +46,12 @@ public List getHandlers(HiveConf conf, TxnStore txnHandler, Metadat boolean metricsEnabled, FSRemover fsRemover) { boolean useAbortHandler = MetastoreConf.getBoolVar(conf, MetastoreConf.ConfVars.COMPACTOR_CLEAN_ABORTS_USING_CLEANER); List taskHandlers = new ArrayList<>(); + taskHandlers.add(new CompactionCleaner(conf, txnHandler, metadataCache, + metricsEnabled, fsRemover)); if (useAbortHandler) { taskHandlers.add(new AbortedTxnCleaner(conf, txnHandler, metadataCache, metricsEnabled, fsRemover)); } - taskHandlers.add(new CompactionCleaner(conf, txnHandler, metadataCache, - metricsEnabled, fsRemover)); return taskHandlers; } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 8acaadfa4b04..415d708ab359 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -332,7 +332,7 @@ public void testAbortedCleaningWithThreeTxnsWithDiffWriteIds() throws Exception @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testAbortCleanupNotUpdatingCompactionTables(boolean isPartitioned) throws Exception { + public void testAbortCleanupNotUpdatingSpecificCompactionTables(boolean isPartitioned) throws Exception { String dbName = "default", tableName = "abort_cleanup_not_populating_compaction_tables_test", partName = "today"; Table t = newTable(dbName, tableName, isPartitioned); Partition p = isPartitioned ? newPartition(t, partName) : null; @@ -351,6 +351,7 @@ public void testAbortCleanupNotUpdatingCompactionTables(boolean isPartitioned) t false, mockedFSRemover)); runInitiator(conf); + // Initiator must not add anything to compaction_queue String compactionQueuePresence = "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); @@ -409,15 +410,15 @@ public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { cleaner.run(); Mockito.verify(mockedTxnHandler, Mockito.times(1)).setAbortCleanerRetryRetentionTimeOnError(any(AbortTxnRequestInfo.class)); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); } @@ -455,14 +456,14 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -477,7 +478,7 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); } @ParameterizedTest @@ -509,15 +510,15 @@ public void testRetryWithinRetentionTime(boolean isPartitioned) throws Exception cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -530,7 +531,7 @@ public void testRetryWithinRetentionTime(boolean isPartitioned) throws Exception cleaner.run(); // The retry entry is not removed since retry conditions are not achieved hence its not picked for cleanup. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -572,14 +573,14 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -593,7 +594,7 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.run(); // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -635,14 +636,14 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName + "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery) .replace("\n", "").trim().contains("Testing first retry")); @@ -660,7 +661,7 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.run(); // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); @@ -699,14 +700,14 @@ public void testZeroRetryRetentionTimeForAbortCleanup(boolean isPartitioned) thr cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - String whereClause = " WHERE \"TCQ_DATABASE\" = '" + dbName+ "' AND \"TCQ_TABLE\" = '" + tableName + "' AND \"TCQ_PARTITION\"" + - (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL"); - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); - String retryRetentionQuery = "SELECT \"TCQ_RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; + Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Integer.toString(0), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"TCQ_ERROR_MESSAGE\" as varchar(100)) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause; + String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) .replace("\n", "").trim().contains("Testing retry")); @@ -719,6 +720,6 @@ public void testZeroRetryRetentionTimeForAbortCleanup(boolean isPartitioned) thr cleaner.run(); // The retry entry should be removed since retry conditions are achieved because retry retention time is 0. - Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"TXN_CLEANUP_QUEUE\" " + whereClause)); + Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index 38ab1894ebcc..01b19df64ce7 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -239,14 +239,16 @@ std::string to_string(const LockType::type& val) { int _kCompactionTypeValues[] = { CompactionType::MINOR, CompactionType::MAJOR, - CompactionType::REBALANCE + CompactionType::REBALANCE, + CompactionType::ABORT_CLEANUP }; const char* _kCompactionTypeNames[] = { "MINOR", "MAJOR", - "REBALANCE" + "REBALANCE", + "ABORT_CLEANUP" }; -const std::map _CompactionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(3, _kCompactionTypeValues, _kCompactionTypeNames), ::apache::thrift::TEnumIterator(-1, nullptr, nullptr)); +const std::map _CompactionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kCompactionTypeValues, _kCompactionTypeNames), ::apache::thrift::TEnumIterator(-1, nullptr, nullptr)); std::ostream& operator<<(std::ostream& out, const CompactionType::type& val) { std::map::const_iterator it = _CompactionType_VALUES_TO_NAMES.find(val); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 3223c3340354..93f099bef108 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -127,7 +127,8 @@ struct CompactionType { enum type { MINOR = 1, MAJOR = 2, - REBALANCE = 3 + REBALANCE = 3, + ABORT_CLEANUP = 4 }; }; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java index bfa7b27dd66c..a7efdff16787 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java @@ -11,7 +11,8 @@ public enum CompactionType implements org.apache.thrift.TEnum { MINOR(1), MAJOR(2), - REBALANCE(3); + REBALANCE(3), + ABORT_CLEANUP(4); private final int value; @@ -39,6 +40,8 @@ public static CompactionType findByValue(int value) { return MAJOR; case 3: return REBALANCE; + case 4: + return ABORT_CLEANUP; default: return null; } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php index 9c62a8e4696a..688130711d39 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php @@ -24,10 +24,13 @@ final class CompactionType const REBALANCE = 3; + const ABORT_CLEANUP = 4; + static public $__names = array( 1 => 'MINOR', 2 => 'MAJOR', 3 => 'REBALANCE', + 4 => 'ABORT_CLEANUP', ); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index 50a6c5fe8088..53db15c9a5ef 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -156,17 +156,20 @@ class CompactionType(object): MINOR = 1 MAJOR = 2 REBALANCE = 3 + ABORT_CLEANUP = 4 _VALUES_TO_NAMES = { 1: "MINOR", 2: "MAJOR", 3: "REBALANCE", + 4: "ABORT_CLEANUP", } _NAMES_TO_VALUES = { "MINOR": 1, "MAJOR": 2, "REBALANCE": 3, + "ABORT_CLEANUP": 4, } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index 9d864db0d059..6f014cf0466c 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -71,8 +71,9 @@ module CompactionType MINOR = 1 MAJOR = 2 REBALANCE = 3 - VALUE_MAP = {1 => "MINOR", 2 => "MAJOR", 3 => "REBALANCE"} - VALID_VALUES = Set.new([MINOR, MAJOR, REBALANCE]).freeze + ABORT_CLEANUP = 4 + VALUE_MAP = {1 => "MINOR", 2 => "MAJOR", 3 => "REBALANCE", 4 => "ABORT_CLEANUP"} + VALID_VALUES = Set.new([MINOR, MAJOR, REBALANCE, ABORT_CLEANUP]).freeze end module GrantRevokeType diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index 63936d299850..0113c24ad9da 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -202,6 +202,7 @@ enum CompactionType { MINOR = 1, MAJOR = 2, REBALANCE = 3, + ABORT_CLEANUP = 4, } enum GrantRevokeType { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 0cee8f121c6c..ac69ca33d4fa 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -51,6 +51,7 @@ import java.util.concurrent.TimeUnit; import static org.apache.hadoop.hive.metastore.txn.TxnUtils.getEpochFn; +import static org.apache.hadoop.hive.metastore.txn.TxnUtils.thriftCompactionType2DbType; /** * Extends the transaction handler with methods needed only by the compactor threads. These @@ -94,37 +95,50 @@ class CompactionTxnHandler extends TxnHandler { "AND (\"TXN_STATE\" = " + TxnStatus.ABORTED + " OR \"TXN_STATE\" = " + TxnStatus.COMMITTED + ") " + "AND \"TXN_ID\" < ?"; - // Checks presence of the retry entry (by assigning 0 in retry retention) for the corresponding table/partition in the TXN_CLEANUP_QUEUE - private static final String SELECT_TXN_CLEANUP_QUEUE_PRESENCE_QUERY = - " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", " + - "-1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + - "-1 AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - "GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\""; - - private static final String SELECT_RETRY_RETENTION_QUERY = - " SELECT \"TCQ_DATABASE\" AS \"DB\", \"TCQ_TABLE\" AS \"TBL\", \"TCQ_PARTITION\" AS \"PART\", " + - " -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + - " MAX(\"TCQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\" FROM \"TXN_CLEANUP_QUEUE\" " + - " GROUP BY \"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\" " + - "HAVING MIN(\"TCQ_RETRY_TIME\") < %s - MAX(\"TCQ_RETRY_RETENTION\")"; - - private static final String SELECT_MIN_OPEN_WRITETXN_ID_QUERY = - " SELECT MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + - " FROM \"TXNS\", \"TXN_COMPONENTS\" WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\"=" + TxnStatus.OPEN + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\""; - - private static final String SELECT_ABORTED_TXNS_QUERY = "SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", " + - " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + + // Three inner sub-queries which are under union all and gets the max values for corresponding db, table, partition. + private static final String SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY = + "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), " + + " MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\"), MAX(\"RETRY_RECORD_CHECK\") FROM ( " + + // First sub-query - Gets the aborted txns with min txn start time, number of aborted txns + // for corresponding db, table, partition. + " SELECT \"TC_DATABASE\" AS \"DB\", \"TC_TABLE\" AS \"TBL\", \"TC_PARTITION\" AS \"PART\", " + + " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\", " + + " 0 AS \"MIN_OPEN_WRITE_TXNID\", -1 AS \"RETRY_RETENTION\", 0 AS \"RETRY_RECORD_CHECK\" " + + " FROM \"TXNS\", \"TXN_COMPONENTS\" " + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" "; - - private static final String SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY = "SELECT \"tc\".\"TC_DATABASE\" AS \"DB\", \"tc\".\"TC_TABLE\" AS \"TBL\", " + - " \"tc\".\"TC_PARTITION\" AS \"PART\", \"tc\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", " + - " \"tc\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", \"minOpenWriteTxnId\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", -2 AS \"RETRY_RETENTION\" " + - " FROM ( " + SELECT_ABORTED_TXNS_QUERY + " %s ) \"tc\" LEFT JOIN ( " + SELECT_MIN_OPEN_WRITETXN_ID_QUERY + " ) \"minOpenWriteTxnId\" " + - " ON \"tc\".\"TC_DATABASE\" = \"minOpenWriteTxnId\".\"TC_DATABASE\" AND \"tc\".\"TC_TABLE\" = \"minOpenWriteTxnId\".\"TC_TABLE\"" + - " AND (\"tc\".\"TC_PARTITION\" = \"minOpenWriteTxnId\".\"TC_PARTITION\" OR (\"tc\".\"TC_PARTITION\" IS NULL AND \"minOpenWriteTxnId\".\"TC_PARTITION\" IS NULL))"; - + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" %s " + + " UNION ALL " + + // Second sub-query - Gets the min open txn id for corresponding db, table, partition. + " SELECT \"TC_DATABASE\" AS \"DB\", \"TC_TABLE\" AS \"TBL\", \"TC_PARTITION\" AS \"PART\", " + + " 0 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", " + + " -1 AS \"RETRY_RETENTION\", 0 AS \"RETRY_RECORD_CHECK\" " + + " FROM \"TXNS\", \"TXN_COMPONENTS\" " + + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.OPEN + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + + " UNION ALL " + + // Third sub-query - Gets the retry entries for corresponding db, table, partition. + " SELECT \"CQ_DATABASE\" AS \"DB\", \"CQ_TABLE\" AS \"TBL\", \"CQ_PARTITION\" AS \"PART\", " + + " -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + + " MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\", " + + " MIN(\"CQ_COMMIT_TIME\") - %s + MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RECORD_CHECK\" FROM \"COMPACTION_QUEUE\" " + + " WHERE \"CQ_TYPE\" = " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + + " GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\" " + + ") res GROUP BY \"DB\", \"TBL\", \"PART\""; + + private static final String DELETE_ABORT_RETRY_ENTRIES_FROM_COMPACTION_QUEUE = + "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\" = ? " + + "AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) AND \"CQ_TYPE\" = " + + quoteChar(TxnStore.ABORT_CLEANUP_TYPE); + + private static final String INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE = + "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", " + + " \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", \"CQ_TYPE\", \"CQ_ID\", \"CQ_STATE\", \"CQ_COMMIT_TIME\") " + + " VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s)"; + + private static final String UPDATE_ABORT_RETRY_ENTRY_IN_COMPACTION_QUEUE = + "UPDATE \"COMPACTION_QUEUE\" SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\" = ? " + + " WHERE \"CQ_DATABASE\" = ? AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) " + + " AND \"CQ_TYPE\" = ?"; public CompactionTxnHandler() { } @@ -289,7 +303,9 @@ public CompactionInfo findNextToCompact(FindNextCompactRequest rqst) throws Meta StringBuilder sb = new StringBuilder(); sb.append("SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + "\"CQ_TYPE\", \"CQ_POOL_NAME\", \"CQ_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\", " + - "\"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = '" + INITIATED_STATE + "' AND "); + "\"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" " + + "WHERE \"CQ_TYPE\" != " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + + " AND \"CQ_STATE\" = " + quoteChar(INITIATED_STATE) + " AND "); boolean hasPoolName = StringUtils.isNotBlank(rqst.getPoolName()); if(hasPoolName) { sb.append("\"CQ_POOL_NAME\"=?"); @@ -433,7 +449,8 @@ public List findReadyToClean(long minOpenTxnWaterMark, long rete * By filtering on minOpenTxnWaterMark, we will only cleanup after every transaction is committed, that could see * the uncompacted deltas. This way the cleaner can clean up everything that was made obsolete by this compaction. */ - String whereClause = " WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING) + + String whereClause = " WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING) + + " AND \"CQ_TYPE\" != " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + " AND (\"CQ_COMMIT_TIME\" < (" + getEpochFn(dbProduct) + " - \"CQ_RETRY_RETENTION\" - " + retentionTime + ") OR \"CQ_COMMIT_TIME\" IS NULL)"; String queryStr = @@ -512,17 +529,10 @@ public List findReadyToCleanAborts(long abortedTimeThreshol try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction); Statement stmt = dbConn.createStatement()) { boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0; - String firstInnerQuery = String.format(SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY, - checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold); - - String secondInnerQuery = String.format(SELECT_RETRY_RETENTION_QUERY, getEpochFn(dbProduct)); - - String unionQuery = firstInnerQuery + " UNION ALL " + secondInnerQuery + " UNION ALL " + SELECT_TXN_CLEANUP_QUEUE_PRESENCE_QUERY; + String sCheckAborted = String.format(SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY, + checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold, getEpochFn(dbProduct)); - String sCheckAborted = "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), " + - "MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\") FROM ( " + unionQuery + " ) res GROUP BY \"DB\", \"TBL\", \"PART\""; - - LOG.debug("Going to execute query <{}>", sCheckAborted); + LOG.info("Going to execute query <{}>", sCheckAborted); try (ResultSet rs = stmt.executeQuery(sCheckAborted)) { long systemTime = System.currentTimeMillis(); while (rs.next()) { @@ -535,14 +545,13 @@ public List findReadyToCleanAborts(long abortedTimeThreshol info.tableName = rs.getString(2); info.partName = rs.getString(3); // In this case, this field contains min open write txn ID. - info.minOpenWriteTxnId = rs.getLong(6); - // The query returns the following retry retention values - 3 cases exist - - // -2 - No retry entry in TXN_CLEANUP_QUEUE - // -1 - Retry entry is present but hasn't achieved conditions for retry (i.e. min(tcq_retry_time) >= current_timestamp - max(retry_retention_time)) - // >= 0 - Retry entry is present and has achieved conditions for retry (i.e. min(tcq_retry_time) < current_timestamp - max(retry_retention_time)) + info.minOpenWriteTxnId = rs.getLong(6) > 0 ? rs.getLong(6) : Long.MAX_VALUE; + // The specific type, state assigned to abort cleanup. + info.type = CompactionType.ABORT_CLEANUP; + info.state = READY_FOR_CLEANING; info.retryRetention = rs.getLong(7); - // Do not add entry when retry retention is -1. - if (info.retryRetention != -1) { + // Do not add entry when retry record check is > 0. + if (rs.getLong(8) <= 0) { // In this case, this flag is used for checking whether a retry entry needs to be inserted or updated. info.hasRetryEntry = info.retryRetention >= 0; readyToCleanAborts.add(info); @@ -751,11 +760,11 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws ResultSet rs = null; try { /* - * Remove all retry associated metadata of table/partition in the TXN_CLEANUP_QUEUE both when compaction - * or abort cleanup is successful. We dont want a situation wherein we have a retry queue entry for a table + * Remove all abort retry associated metadata of table/partition in the COMPACTION_QUEUE both when compaction + * or abort cleanup is successful. We don't want a situation wherein we have a abort retry entry for a table * but no corresponding entry in TXN_COMPONENTS table. */ - TxnCleanupQueueHandler.removeRetryQueueEntries(this, dbConn, info); + removeAbortRetryEntries(dbConn, info); /* * compaction may remove data from aborted txns above tc_writeid bit it only guarantees to @@ -1252,79 +1261,6 @@ private static class RetentionCounters { } } - static class TxnCleanupQueueHandler { - - public static void removeRetryQueueEntries(CompactionTxnHandler txnHandler, - Connection dbConn, CompactionInfo info) throws MetaException, RetryException { - PreparedStatement pStmt = null; - String query = "DELETE FROM \"TXN_CLEANUP_QUEUE\" WHERE \"TCQ_DATABASE\" = ? " + - "AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; - try { - LOG.debug("Going to execute update <{}>", query); - pStmt = dbConn.prepareStatement(query); - pStmt.setString(1, info.dbname); - pStmt.setString(2, info.tableName); - if (info.partName != null) { - pStmt.setString(3, info.partName); - } else { - // Since the type of 'TCQ_PARTITION' column is varchar. - // Hence, setting null for VARCHAR type. - pStmt.setNull(3, Types.VARCHAR); - } - int rc = pStmt.executeUpdate(); - LOG.debug("Removed {} records in txn_cleanup_queue", rc); - } catch (SQLException e) { - LOG.error("Unable to delete from txn components due to {}", e.getMessage()); - LOG.debug("Going to rollback"); - rollbackDBConn(dbConn); - txnHandler.checkRetryable(e, "removeRetryQueueEntries(" + info + ")"); - throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); - } finally { - closeStmt(pStmt); - } - } - - public static void setRetryRetentionTimeOnError(CompactionTxnHandler txnHandler, - Connection dbConn, AbortTxnRequestInfo info) throws MetaException, RetryException { - String query; - if (!info.hasRetryEntry) { - query = "INSERT INTO \"TXN_CLEANUP_QUEUE\" (\"TCQ_RETRY_RETENTION\", \"TCQ_ERROR_MESSAGE\", " + - "\"TCQ_DATABASE\", \"TCQ_TABLE\", \"TCQ_PARTITION\", \"TCQ_RETRY_TIME\") VALUES (?, ?, ?, ?, ?, " + getEpochFn(dbProduct) + ")"; - } else { - query = "UPDATE \"TXN_CLEANUP_QUEUE\" SET \"TCQ_RETRY_RETENTION\" = ?, \"TCQ_ERROR_MESSAGE\" = ? " + - "WHERE \"TCQ_DATABASE\" = ? AND \"TCQ_TABLE\" = ? AND (\"TCQ_PARTITION\" = ? OR \"TCQ_PARTITION\" IS NULL)"; - } - try (PreparedStatement stmt = dbConn.prepareStatement(query)) { - stmt.setLong(1, info.retryRetention); - stmt.setString(2, info.errorMessage); - stmt.setString(3, info.dbname); - stmt.setString(4, info.tableName); - if (info.partName != null) { - stmt.setString(5, info.partName); - } else { - // Since the type of 'TCQ_PARTITION' column is varchar. - // Hence, setting null for VARCHAR type. - stmt.setNull(5, Types.VARCHAR); - } - int updCnt = stmt.executeUpdate(); - if (updCnt == 0) { - LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); - dbConn.rollback(); - throw new MetaException("No record with TCQ_DATABASE=" + info.dbname + ", TCQ_TABLE=" - + info.tableName + ", TCQ_PARTITION" + info.partName + " found in TXN_CLEANUP_QUEUE"); - } - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.error("Unable to update compaction queue: " + e.getMessage()); - rollbackDBConn(dbConn); - txnHandler.checkRetryable(e, "setCleanerRetryRetentionTimeOnError(" + info + ")"); - throw new MetaException("Unable to update compaction queue: " + - e.getMessage()); - } - } - } - private void checkForDeletion(List deleteSet, CompactionInfo ci, RetentionCounters rc, long timeoutThreshold) { switch (ci.state) { case DID_NOT_INITIATE: @@ -1362,6 +1298,73 @@ private static boolean timedOut(CompactionInfo ci, RetentionCounters rc, long pa && (rc.hasSucceededMajorCompaction || (rc.hasSucceededMinorCompaction && ci.type == CompactionType.MINOR)); } + private void removeAbortRetryEntries(Connection dbConn, CompactionInfo info) throws MetaException, RetryException { + LOG.debug("Going to execute update <{}>", DELETE_ABORT_RETRY_ENTRIES_FROM_COMPACTION_QUEUE); + try (PreparedStatement pStmt = dbConn.prepareStatement(DELETE_ABORT_RETRY_ENTRIES_FROM_COMPACTION_QUEUE)) { + pStmt.setString(1, info.dbname); + pStmt.setString(2, info.tableName); + if (info.partName != null) { + pStmt.setString(3, info.partName); + } else { + // Since the type of 'CQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + pStmt.setNull(3, Types.VARCHAR); + } + int rc = pStmt.executeUpdate(); + LOG.debug("Removed {} records in COMPACTION_QUEUE", rc); + } catch (SQLException e) { + LOG.error("Unable to delete abort retry entries from COMPACTION_QUEUE due to {}", e.getMessage()); + LOG.debug("Going to rollback"); + rollbackDBConn(dbConn); + checkRetryable(e, "removeAbortRetryEntries(" + info + ")"); + throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); + } + } + + private void setAbortRetryRetentionTimeOnError(Connection dbConn, AbortTxnRequestInfo info) throws MetaException, RetryException { + String query = info.hasRetryEntry ? UPDATE_ABORT_RETRY_ENTRY_IN_COMPACTION_QUEUE : + String.format(INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE, getEpochFn(dbProduct)); + try (PreparedStatement pStmt = dbConn.prepareStatement(query); Statement stmt = dbConn.createStatement()) { + pStmt.setLong(1, info.retryRetention); + pStmt.setString(2, info.errorMessage); + pStmt.setString(3, info.dbname); + pStmt.setString(4, info.tableName); + if (info.partName != null) { + pStmt.setString(5, info.partName); + } else { + // Since the type of 'CQ_PARTITION' column is varchar. + // Hence, setting null for VARCHAR type. + pStmt.setNull(5, Types.VARCHAR); + } + pStmt.setString(6, Character.toString(thriftCompactionType2DbType(info.type))); + if (!info.hasRetryEntry) { + long id = generateCompactionQueueId(stmt); + pStmt.setLong(7, id); + pStmt.setString(8, Character.toString(info.state)); + } + int updCnt = pStmt.executeUpdate(); + if (updCnt == 0) { + LOG.error("Unable to update/insert compaction queue record: {}. updCnt={}", info, updCnt); + dbConn.rollback(); + if (info.hasRetryEntry) { + throw new MetaException("No abort retry record with CQ_DATABASE=" + info.dbname + ", CQ_TABLE=" + + info.tableName + ", CQ_PARTITION" + info.partName + " found in COMPACTION_QUEUE"); + } else { + throw new MetaException("Unable to insert abort retry entry into COMPACTION QUEUE: " + + " CQ_DATABASE=" + info.dbname + ", CQ_TABLE=" + info.tableName + ", CQ_PARTITION" + info.partName); + } + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + LOG.error("Unable to update compaction queue: " + e.getMessage()); + rollbackDBConn(dbConn); + checkRetryable(e, "setAbortRetryRetentionTimeOnError(" + info + ")"); + throw new MetaException("Unable to update compaction queue: " + + e.getMessage()); + } + } + /** * For any given compactable entity (partition; table if not partitioned) the history of compactions * may look like "sssfffaaasffss", for example. The idea is to retain the tail (most recent) of the @@ -1652,7 +1655,7 @@ public void markRefused(CompactionInfo info) throws MetaException { public void setAbortCleanerRetryRetentionTimeOnError(AbortTxnRequestInfo info) throws MetaException { try { try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { - TxnCleanupQueueHandler.setRetryRetentionTimeOnError(this, dbConn, info); + setAbortRetryRetentionTimeOnError(dbConn, info); } catch (SQLException e) { LOG.error(DB_FAILED_TO_CONNECT + e.getMessage()); checkRetryable(e, "setAbortCleanerRetryRetentionTimeOnError(" + info + ")"); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index 2bb55ee12fc7..5c919b4df10e 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -124,6 +124,7 @@ enum MUTEX_KEY { char MAJOR_TYPE = 'a'; char MINOR_TYPE = 'i'; char REBALANCE_TYPE = 'r'; + char ABORT_CLEANUP_TYPE = 'c'; String[] COMPACTION_STATES = new String[] {INITIATED_RESPONSE, WORKING_RESPONSE, CLEANING_RESPONSE, FAILED_RESPONSE, SUCCEEDED_RESPONSE, DID_NOT_INITIATE_RESPONSE, REFUSED_RESPONSE }; diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index 704956a6f460..854d7377318e 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -586,6 +586,8 @@ public static CompactionType dbCompactionType2ThriftType(char dbValue) throws Me return CompactionType.MINOR; case TxnStore.REBALANCE_TYPE: return CompactionType.REBALANCE; + case TxnStore.ABORT_CLEANUP_TYPE: + return CompactionType.ABORT_CLEANUP; default: throw new MetaException("Unexpected compaction type " + dbValue); } @@ -599,6 +601,8 @@ public static Character thriftCompactionType2DbType(CompactionType ct) throws Me return TxnStore.MINOR_TYPE; case REBALANCE: return TxnStore.REBALANCE_TYPE; + case ABORT_CLEANUP: + return TxnStore.ABORT_CLEANUP_TYPE; default: throw new MetaException("Unexpected compaction type " + ct); } diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql index 03b94b4a6bae..68f9dbe08635 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/hive-schema-4.0.0.derby.sql @@ -640,15 +640,6 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ); -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE clob -); - CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( NCQ_NEXT bigint NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql index 6d99f37c9150..afd38a400319 100644 --- a/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql +++ b/standalone-metastore/metastore-server/src/main/sql/derby/upgrade-4.0.0-alpha-2-to-4.0.0.derby.sql @@ -26,15 +26,5 @@ CREATE INDEX "APP"."TAB_COL_STATS_IDX" ON "APP"."TAB_COL_STATS" ("DB_NAME", "TAB DROP INDEX "APP"."PCS_STATS_IDX"; CREATE INDEX "APP"."PCS_STATS_IDX" ON "APP"."PART_COL_STATS" ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME","CAT_NAME"); --- HIVE-27332 -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE clob -); - -- This needs to be the last thing done. Insert any changes above this line. UPDATE "APP".VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; \ No newline at end of file diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql index 05344b5ce116..df643b7432a8 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/hive-schema-4.0.0.mssql.sql @@ -1063,15 +1063,6 @@ PRIMARY KEY CLUSTERED ) ); -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint NULL, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE varchar(max) NULL -); - CREATE TABLE COMPLETED_COMPACTIONS ( CC_ID bigint NOT NULL, CC_DATABASE nvarchar(128) NOT NULL, diff --git a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql index bece0e64a7a0..235b52ee741f 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mssql/upgrade-4.0.0-alpha-2-to-4.0.0.mssql.sql @@ -26,16 +26,6 @@ CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PAR DROP INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS; CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAME, CAT_NAME); --- HIVE-27332 -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint NULL, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE varchar(max) NULL -); - -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql index 3da816a7efeb..99a12998f10f 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/hive-schema-4.0.0.mysql.sql @@ -1101,15 +1101,6 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ) ENGINE=InnoDB DEFAULT CHARSET=latin1; -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE mediumtext -) ENGINE=InnoDB DEFAULT CHARSET=latin1; - CREATE TABLE COMPLETED_COMPACTIONS ( CC_ID bigint PRIMARY KEY, CC_DATABASE varchar(128) NOT NULL, diff --git a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql index 6191e7a43413..67cede7d8c6d 100644 --- a/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql +++ b/standalone-metastore/metastore-server/src/main/sql/mysql/upgrade-4.0.0-alpha-2-to-4.0.0.mysql.sql @@ -27,16 +27,6 @@ CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAM DROP INDEX PCS_STATS_IDX ON PART_COL_STATS; CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME,CAT_NAME) USING BTREE; --- HIVE-27332 -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME bigint, - TCQ_RETRY_RETENTION bigint NOT NULL DEFAULT 0, - TCQ_ERROR_MESSAGE mediumtext -) ENGINE=InnoDB DEFAULT CHARSET=latin1; - -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS MESSAGE; diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql index d75c97aa3290..b9c98928046f 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/hive-schema-4.0.0.oracle.sql @@ -1103,15 +1103,6 @@ CREATE TABLE COMPACTION_QUEUE ( CQ_ORDER_BY varchar(4000) ) ROWDEPENDENCIES; -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME NUMBER(19), - TCQ_RETRY_RETENTION NUMBER(19) DEFAULT 0 NOT NULL, - TCQ_ERROR_MESSAGE CLOB -); - CREATE TABLE NEXT_COMPACTION_QUEUE_ID ( NCQ_NEXT NUMBER(19) NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql index c16679eabf89..c33e400e460c 100644 --- a/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql +++ b/standalone-metastore/metastore-server/src/main/sql/oracle/upgrade-4.0.0-alpha-2-to-4.0.0.oracle.sql @@ -26,16 +26,6 @@ CREATE INDEX TAB_COL_STATS_IDX ON TAB_COL_STATS (DB_NAME, TABLE_NAME, COLUMN_NAM DROP INDEX PCS_STATS_IDX; CREATE INDEX PCS_STATS_IDX ON PART_COL_STATS (DB_NAME,TABLE_NAME,COLUMN_NAME,PARTITION_NAME,CAT_NAME); --- HIVE-27332 -CREATE TABLE TXN_CLEANUP_QUEUE ( - TCQ_DATABASE varchar(128) NOT NULL, - TCQ_TABLE varchar(256) NOT NULL, - TCQ_PARTITION varchar(767), - TCQ_RETRY_TIME NUMBER(19), - TCQ_RETRY_RETENTION NUMBER(19) DEFAULT 0 NOT NULL, - TCQ_ERROR_MESSAGE CLOB -); - -- These lines need to be last. Insert any changes above. UPDATE VERSION SET SCHEMA_VERSION='4.0.0', VERSION_COMMENT='Hive release version 4.0.0' where VER_ID=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0' AS Status from dual; diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql index 26d1f7f76e45..032989a3302b 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/hive-schema-4.0.0.postgres.sql @@ -1818,15 +1818,6 @@ CREATE TABLE "COMPACTION_QUEUE" ( "CQ_ORDER_BY" varchar(4000) ); -CREATE TABLE "TXN_CLEANUP_QUEUE" ( - "TCQ_DATABASE" varchar(128) NOT NULL, - "TCQ_TABLE" varchar(256) NOT NULL, - "TCQ_PARTITION" varchar(767), - "TCQ_RETRY_TIME" bigint, - "TCQ_RETRY_RETENTION" bigint NOT NULL DEFAULT 0, - "TCQ_ERROR_MESSAGE" text -); - CREATE TABLE "NEXT_COMPACTION_QUEUE_ID" ( "NCQ_NEXT" bigint NOT NULL ); diff --git a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql index 8a9ed46565cc..041d9f55032a 100644 --- a/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql +++ b/standalone-metastore/metastore-server/src/main/sql/postgres/upgrade-4.0.0-alpha-2-to-4.0.0.postgres.sql @@ -26,16 +26,6 @@ CREATE INDEX "TAB_COL_STATS_IDX" ON "TAB_COL_STATS" USING btree ("DB_NAME","TABL DROP INDEX "PCS_STATS_IDX"; CREATE INDEX "PCS_STATS_IDX" ON "PART_COL_STATS" USING btree ("DB_NAME","TABLE_NAME","COLUMN_NAME","PARTITION_NAME","CAT_NAME"); --- HIVE-27332 -CREATE TABLE "TXN_CLEANUP_QUEUE" ( - "TCQ_DATABASE" varchar(128) NOT NULL, - "TCQ_TABLE" varchar(256) NOT NULL, - "TCQ_PARTITION" varchar(767), - "TCQ_RETRY_TIME" bigint, - "TCQ_RETRY_RETENTION" bigint NOT NULL DEFAULT 0, - "TCQ_ERROR_MESSAGE" text -); - -- These lines need to be last. Insert any changes above. UPDATE "VERSION" SET "SCHEMA_VERSION"='4.0.0', "VERSION_COMMENT"='Hive release version 4.0.0' where "VER_ID"=1; SELECT 'Finished upgrading MetaStore schema from 4.0.0-alpha-2 to 4.0.0'; diff --git a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java index 6a2a3c9a0bf9..9e712a9c54b6 100644 --- a/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java +++ b/standalone-metastore/metastore-server/src/test/java/org/apache/hadoop/hive/metastore/utils/TestTxnDbUtil.java @@ -223,7 +223,6 @@ public static void cleanDb(Configuration conf) throws Exception { success &= truncateTable(conn, conf, stmt, "HIVE_LOCKS"); success &= truncateTable(conn, conf, stmt, "NEXT_LOCK_ID"); success &= truncateTable(conn, conf, stmt, "COMPACTION_QUEUE"); - success &= truncateTable(conn, conf, stmt, "TXN_CLEANUP_QUEUE"); success &= truncateTable(conn, conf, stmt, "NEXT_COMPACTION_QUEUE_ID"); success &= truncateTable(conn, conf, stmt, "COMPLETED_COMPACTIONS"); success &= truncateTable(conn, conf, stmt, "AUX_TABLE"); From 3f9113bf10d122988f3ca4167beb1cb878416b80 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Sun, 4 Jun 2023 10:15:03 +0530 Subject: [PATCH 5/6] Address review comments - 4 --- .../hive/ql/txn/compactor/TestCompactor.java | 3 +- .../compactor/handler/AbortedTxnCleaner.java | 34 +-- .../compactor/handler/CompactionCleaner.java | 31 +-- .../ql/txn/compactor/handler/TaskHandler.java | 29 +++ .../txn/TestCompactionTxnHandler.java | 10 +- .../hive/ql/txn/compactor/TestCleaner.java | 5 +- .../handler/TestAbortedTxnCleaner.java | 121 ++++++---- .../thrift/gen-cpp/hive_metastore_types.cpp | 4 +- .../gen/thrift/gen-cpp/hive_metastore_types.h | 2 +- .../hive/metastore/api/CompactionType.java | 4 +- .../gen-php/metastore/CompactionType.php | 4 +- .../thrift/gen-py/hive_metastore/ttypes.py | 6 +- .../gen/thrift/gen-rb/hive_metastore_types.rb | 6 +- .../hive/metastore/hive_metastore.proto | 2 + .../src/main/thrift/hive_metastore.thrift | 2 +- .../hadoop/hive/metastore/HMSHandler.java | 2 +- .../metastore/txn/AbortTxnRequestInfo.java | 41 ---- .../hive/metastore/txn/CompactionInfo.java | 4 + .../metastore/txn/CompactionTxnHandler.java | 215 +++++++++--------- .../hadoop/hive/metastore/txn/TxnHandler.java | 4 +- .../hadoop/hive/metastore/txn/TxnStore.java | 22 +- .../hadoop/hive/metastore/txn/TxnUtils.java | 8 +- 22 files changed, 259 insertions(+), 300 deletions(-) delete mode 100644 standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java diff --git a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java index dead15e1799c..7083df59828c 100644 --- a/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java +++ b/itests/hive-unit/src/test/java/org/apache/hadoop/hive/ql/txn/compactor/TestCompactor.java @@ -84,7 +84,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assume.assumeTrue; import static org.mockito.ArgumentMatchers.any; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.times; @@ -1128,7 +1127,7 @@ public void testAbortAfterMarkCleaned() throws Exception { doAnswer(invocationOnMock -> { connection2.abortTransaction(); return invocationOnMock.callRealMethod(); - }).when(mockedTxnHandler).markCleaned(any(), eq(false)); + }).when(mockedTxnHandler).markCleaned(any()); MetadataCache metadataCache = new MetadataCache(false); FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index 510ee0f3aaae..65e7cf71b9cf 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -25,7 +25,7 @@ import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.metrics.MetricsConstants; import org.apache.hadoop.hive.metastore.metrics.PerfLogger; -import org.apache.hadoop.hive.metastore.txn.AbortTxnRequestInfo; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.txn.TxnUtils; import org.apache.hadoop.hive.ql.txn.compactor.CompactorUtil; @@ -41,8 +41,6 @@ import java.util.stream.Collectors; import static java.util.Objects.isNull; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; /** * Abort-cleanup based implementation of TaskHandler. @@ -78,17 +76,17 @@ public List getTasks() throws MetaException { long abortedTimeThreshold = HiveConf .getTimeVar(conf, HiveConf.ConfVars.HIVE_COMPACTOR_ABORTEDTXN_TIME_THRESHOLD, TimeUnit.MILLISECONDS); - List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); + List readyToCleanAborts = txnHandler.findReadyToCleanAborts(abortedTimeThreshold, abortedThreshold); if (!readyToCleanAborts.isEmpty()) { return readyToCleanAborts.stream().map(info -> ThrowingRunnable.unchecked(() -> - clean(info, info.minOpenWriteTxnId, metricsEnabled))) + clean(info, info.txnId, metricsEnabled))) .collect(Collectors.toList()); } return Collections.emptyList(); } - private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metricsEnabled) throws MetaException, InterruptedException { + private void clean(CompactionInfo info, long minOpenWriteTxn, boolean metricsEnabled) throws MetaException, InterruptedException { LOG.info("Starting cleaning for {}", info); PerfLogger perfLogger = PerfLogger.getPerfLogger(false); String cleanerMetric = MetricsConstants.COMPACTION_CLEANER_CYCLE + "_"; @@ -101,7 +99,7 @@ private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metri if (isNull(t)) { // The table was dropped before we got around to cleaning it. LOG.info("Unable to find table {}, assuming it was dropped.", info.getFullTableName()); - txnHandler.markCleaned(info, true); + txnHandler.markCleaned(info); return; } if (!isNull(info.partName)) { @@ -110,7 +108,7 @@ private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metri // The partition was dropped before we got around to cleaning it. LOG.info("Unable to find partition {}, assuming it was dropped.", info.getFullPartitionName()); - txnHandler.markCleaned(info, true); + txnHandler.markCleaned(info); return; } } @@ -122,14 +120,12 @@ private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metri } catch (InterruptedException e) { LOG.error("Caught an interrupted exception when cleaning, unable to complete cleaning of {} due to {}", info, e.getMessage()); - info.errorMessage = e.getMessage(); - handleCleanerAttemptFailure(info); + handleCleanerAttemptFailure(info, e.getMessage()); throw e; } catch (Exception e) { LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", info, e.getMessage()); - info.errorMessage = e.getMessage(); - handleCleanerAttemptFailure(info); + handleCleanerAttemptFailure(info, e.getMessage()); throw new MetaException(e.getMessage()); } finally { if (metricsEnabled) { @@ -138,7 +134,7 @@ private void clean(AbortTxnRequestInfo info, long minOpenWriteTxn, boolean metri } } - private void abortCleanUsingAcidDir(AbortTxnRequestInfo info, String location, long minOpenWriteTxn) throws Exception { + private void abortCleanUsingAcidDir(CompactionInfo info, String location, long minOpenWriteTxn) throws Exception { ValidTxnList validTxnList = TxnUtils.createValidTxnListForCleaner(txnHandler.getOpenTxns(), minOpenWriteTxn, true); //save it so that getAcidState() sees it @@ -157,20 +153,10 @@ private void abortCleanUsingAcidDir(AbortTxnRequestInfo info, String location, l boolean success = cleanAndVerifyObsoleteDirectories(info, location, validWriteIdList, table); if (success || CompactorUtil.isDynPartAbort(table, info.partName)) { - txnHandler.markCleaned(info, true); + txnHandler.markCleaned(info); } else { LOG.warn("Leaving aborted entry {} in TXN_COMPONENTS table.", info); } } - - private void handleCleanerAttemptFailure(AbortTxnRequestInfo info) throws MetaException { - long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); - int cleanAttempts = 0; - if (info.retryRetention > 0) { - cleanAttempts = (int)(Math.log(info.retryRetention / defaultRetention) / Math.log(2)) + 1; - } - info.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; - txnHandler.setAbortCleanerRetryRetentionTimeOnError(info); - } } diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java index f7c8944ea3ee..a928f1438f49 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/CompactionCleaner.java @@ -58,10 +58,6 @@ import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETENTION_TIME; import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.HIVE_COMPACTOR_DELAYED_CLEANUP_ENABLED; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; -import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; import static java.util.Objects.isNull; /** @@ -122,7 +118,7 @@ private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) t // The table was dropped before we got around to cleaning it. LOG.info("Unable to find table {}, assuming it was dropped. {}", ci.getFullTableName(), idWatermark(ci)); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); return; } if (MetaStoreUtils.isNoCleanUpSet(t.getParameters())) { @@ -137,7 +133,7 @@ private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) t // The partition was dropped before we got around to cleaning it. LOG.info("Unable to find partition {}, assuming it was dropped. {}", ci.getFullPartitionName(), idWatermark(ci)); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); return; } if (MetaStoreUtils.isNoCleanUpSet(p.getParameters())) { @@ -168,11 +164,10 @@ private void clean(CompactionInfo ci, long minOpenTxn, boolean metricsEnabled) t } catch (Exception e) { LOG.error("Caught exception when cleaning, unable to complete cleaning of {} due to {}", ci, e.getMessage()); - ci.errorMessage = e.getMessage(); if (metricsEnabled) { Metrics.getOrCreateCounter(MetricsConstants.COMPACTION_CLEANER_FAILURE_COUNTER).inc(); } - handleCleanerAttemptFailure(ci); + handleCleanerAttemptFailure(ci, e.getMessage()); } finally { if (metricsEnabled) { perfLogger.perfLogEnd(CompactionCleaner.class.getName(), cleanerMetric); @@ -204,7 +199,7 @@ private void cleanUsingLocation(CompactionInfo ci, String path, boolean requires deleted = fsRemover.clean(getCleaningRequestBasedOnLocation(ci, path)); } if (!deleted.isEmpty()) { - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); } else { txnHandler.clearCleanerStart(ci); } @@ -253,7 +248,7 @@ private void cleanUsingAcidDir(CompactionInfo ci, String location, long minOpenT boolean success = cleanAndVerifyObsoleteDirectories(ci, location, validWriteIdList, table); if (success || CompactorUtil.isDynPartAbort(table, ci.partName)) { - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); } else { txnHandler.clearCleanerStart(ci); LOG.warn("No files were removed. Leaving queue entry {} in ready for cleaning state.", ci); @@ -302,22 +297,6 @@ protected ValidReaderWriteIdList getValidCleanerWriteIdList(CompactionInfo ci, V return validWriteIdList; } - private void handleCleanerAttemptFailure(CompactionInfo ci) throws MetaException { - long defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); - int cleanAttempts = 0; - if (ci.retryRetention > 0) { - cleanAttempts = (int)(Math.log(ci.retryRetention / defaultRetention) / Math.log(2)) + 1; - } - if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) { - //Mark it as failed if the max attempt threshold is reached. - txnHandler.markFailed(ci); - } else { - //Calculate retry retention time and update record. - ci.retryRetention = (long)Math.pow(2, cleanAttempts) * defaultRetention; - txnHandler.setCleanerRetryRetentionTimeOnError(ci); - } - } - private CleanupRequest getCleaningRequestBasedOnLocation(CompactionInfo ci, String location) { String strIfPurge = ci.getProperty("ifPurge"); boolean ifPurge = strIfPurge != null || Boolean.parseBoolean(ci.getProperty("ifPurge")); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index ef95a100c1a2..aa8b738834ad 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -49,8 +49,13 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; import static org.apache.commons.collections.ListUtils.subtract; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getIntVar; +import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; /** * An abstract class which defines the list of utility methods for performing cleanup activities. @@ -63,6 +68,7 @@ public abstract class TaskHandler { protected final boolean metricsEnabled; protected final MetadataCache metadataCache; protected final FSRemover fsRemover; + protected final long defaultRetention; TaskHandler(HiveConf conf, TxnStore txnHandler, MetadataCache metadataCache, boolean metricsEnabled, FSRemover fsRemover) { @@ -71,6 +77,7 @@ public abstract class TaskHandler { this.metadataCache = metadataCache; this.metricsEnabled = metricsEnabled; this.fsRemover = fsRemover; + this.defaultRetention = getTimeVar(conf, HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS); } public abstract List getTasks() throws MetaException; @@ -161,4 +168,26 @@ protected boolean cleanAndVerifyObsoleteDirectories(CompactionInfo info, String return success; } + + protected void handleCleanerAttemptFailure(CompactionInfo info, String errorMessage) throws MetaException { + int cleanAttempts = 0; + info.errorMessage = errorMessage; + if (info.isAbortedTxnCleanup()) { + info.retryRetention = info.retryRetention > 0 ? info.retryRetention * 2 : defaultRetention; + info.errorMessage = errorMessage; + txnHandler.insertOrSetCleanerRetryRetentionTimeOnError(info); + } else { + if (info.retryRetention > 0) { + cleanAttempts = (int) (Math.log(info.retryRetention / defaultRetention) / Math.log(2)) + 1; + } + if (cleanAttempts >= getIntVar(conf, HIVE_COMPACTOR_CLEANER_MAX_RETRY_ATTEMPTS)) { + //Mark it as failed if the max attempt threshold is reached. + txnHandler.markFailed(info); + } else { + //Calculate retry retention time and update record. + info.retryRetention = (long) Math.pow(2, cleanAttempts) * defaultRetention; + txnHandler.insertOrSetCleanerRetryRetentionTimeOnError(info); + } + } + } } diff --git a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java index 03a7326710ee..5b334838ac8c 100644 --- a/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java +++ b/ql/src/test/org/apache/hadoop/hive/metastore/txn/TestCompactionTxnHandler.java @@ -238,7 +238,7 @@ public void testMarkCleaned() throws Exception { List toClean = txnHandler.findReadyToClean(0, 0); assertEquals(1, toClean.size()); assertNull(txnHandler.findNextToCompact(aFindNextCompactRequest("fred", WORKER_VERSION))); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); assertNull(txnHandler.findNextToCompact(aFindNextCompactRequest("fred", WORKER_VERSION))); assertEquals(0, txnHandler.findReadyToClean(0, 0).size()); @@ -529,7 +529,7 @@ private void addSucceededCompaction(String dbName, String tableName, String part txnHandler.compact(rqst); ci = txnHandler.findNextToCompact(aFindNextCompactRequest("fred", WORKER_VERSION)); assertNotNull(ci); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); } private void addWaitingForCleaningCompaction(String dbName, String tableName, CompactionType type, @@ -866,7 +866,7 @@ public void testMarkCleanedCleansTxnsAndTxnComponents() Thread.sleep(txnHandler.getOpenTxnTimeOutMillis()); List toClean = txnHandler.findReadyToClean(0, 0); assertEquals(1, toClean.size()); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); // Check that we are cleaning up the empty aborted transactions GetOpenTxnsResponse txnList = txnHandler.getOpenTxns(); @@ -892,7 +892,7 @@ public void testMarkCleanedCleansTxnsAndTxnComponents() toClean = txnHandler.findReadyToClean(0, 0); assertEquals(1, toClean.size()); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); txnHandler.openTxns(new OpenTxnRequest(1, "me", "localhost")); // The open txn will became the low water mark @@ -999,7 +999,7 @@ public void testEnqueueTimeThroughLifeCycle() throws Exception { txnHandler.markCompacted(ci); checkEnqueueTime(enqueueTime); - txnHandler.markCleaned(ci, false); + txnHandler.markCleaned(ci); checkEnqueueTime(enqueueTime); } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java index 201b4ee5749f..16266c655bfe 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/TestCleaner.java @@ -67,7 +67,6 @@ import static org.apache.hadoop.hive.metastore.conf.MetastoreConf.getTimeVar; import static org.apache.hadoop.hive.ql.io.AcidUtils.addVisibilitySuffix; import static org.mockito.ArgumentMatchers.anyLong; -import static org.mockito.ArgumentMatchers.eq; import static org.mockito.ArgumentMatchers.nullable; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.doThrow; @@ -108,7 +107,7 @@ public void testRetryAfterFailedCleanup(boolean delayEnabled) throws Exception { FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache); List taskHandlers = TaskHandlerFactory.getInstance() .getHandlers(conf, mockedHandler, metadataCache, false, fsRemover); - doThrow(new RuntimeException(errorMessage)).when(mockedHandler).markCleaned(nullable(CompactionInfo.class), eq(false)); + doThrow(new RuntimeException(errorMessage)).when(mockedHandler).markCleaned(nullable(CompactionInfo.class)); Table t = newTable("default", "retry_test", false); @@ -196,7 +195,7 @@ public void testRetentionAfterFailedCleanup() throws Exception { FSRemover fsRemover = new FSRemover(conf, ReplChangeManager.getInstance(conf), metadataCache); List taskHandlers = TaskHandlerFactory.getInstance() .getHandlers(conf, mockedHandler, metadataCache, false, fsRemover); - doThrow(new RuntimeException()).when(mockedHandler).markCleaned(nullable(CompactionInfo.class), eq(false)); + doThrow(new RuntimeException()).when(mockedHandler).markCleaned(nullable(CompactionInfo.class)); //Do a run to fail the clean and set the retention time Cleaner cleaner = new Cleaner(); diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 415d708ab359..8da8c5b15835 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -24,9 +24,12 @@ import org.apache.hadoop.hive.metastore.api.CompactionRequest; import org.apache.hadoop.hive.metastore.api.CompactionType; import org.apache.hadoop.hive.metastore.api.Partition; +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.Table; import org.apache.hadoop.hive.metastore.conf.MetastoreConf; -import org.apache.hadoop.hive.metastore.txn.AbortTxnRequestInfo; +import org.apache.hadoop.hive.metastore.txn.CompactionInfo; import org.apache.hadoop.hive.metastore.txn.TxnStore; import org.apache.hadoop.hive.metastore.utils.TestTxnDbUtil; import org.apache.hadoop.hive.ql.io.AcidUtils; @@ -409,18 +412,20 @@ public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); cleaner.run(); - Mockito.verify(mockedTxnHandler, Mockito.times(1)).setAbortCleanerRetryRetentionTimeOnError(any(AbortTxnRequestInfo.class)); + Mockito.verify(mockedTxnHandler, Mockito.times(1)).insertOrSetCleanerRetryRetentionTimeOnError(any(CompactionInfo.class)); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) - .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); + .replace("\n", "").trim()); } @ParameterizedTest @@ -456,16 +461,18 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); // Delay for time specified in retry retention. Thread.sleep(retryRetentionTime); @@ -478,7 +485,8 @@ public void testRetryInfoBeingUsed(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + // The retry record must be not present since it will deleted due to successful abort cleanup. + Assert.assertEquals(0, txnHandler.showCompact(new ShowCompactRequest()).getCompactsSize()); } @ParameterizedTest @@ -510,17 +518,19 @@ public void testRetryWithinRetentionTime(boolean isPartitioned) throws Exception cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); Mockito.doAnswer(InvocationOnMock::callRealMethod).when(mockedFSRemover).clean(any()); @@ -531,13 +541,16 @@ public void testRetryWithinRetentionTime(boolean isPartitioned) throws Exception cleaner.run(); // The retry entry is not removed since retry conditions are not achieved hence its not picked for cleanup. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); Assert.assertEquals(Long.toString(MetastoreConf.getTimeVar(conf, MetastoreConf.ConfVars.HIVE_COMPACTOR_CLEANER_RETRY_RETENTION_TIME, TimeUnit.MILLISECONDS)), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); } @ParameterizedTest @@ -573,16 +586,18 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); // Delay for time specified in retry retention. Thread.sleep(retryRetentionTime); @@ -593,14 +608,16 @@ public void testRetryUpdateRetentionTimeWhenFailedTwice(boolean isPartitioned) t cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); } @ParameterizedTest @@ -636,16 +653,18 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing first retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Long.toString(retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery) - .replace("\n", "").trim().contains("Testing first retry")); // Delay for time specified in retry retention. Thread.sleep(retryRetentionTime); @@ -660,14 +679,16 @@ public void testRetryUpdateErrorMessageWhenFailedTwice(boolean isPartitioned) th cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); - // The retry entry is not removed since retry has failed. - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing second retry")); // The retry entry must reflect double the retention time now. Assert.assertEquals(Long.toString(2 * retryRetentionTime), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing second retry")); } @ParameterizedTest @@ -700,16 +721,18 @@ public void testZeroRetryRetentionTimeForAbortCleanup(boolean isPartitioned) thr cleaner.setCleanupHandlers(Arrays.asList(taskHandler)); cleaner.run(); + ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); + Assert.assertEquals(1, scr.getCompactsSize()); + ShowCompactResponseElement scre = scr.getCompacts().get(0); + Assert.assertTrue(scre.getDbname().equals(dbName) && scre.getTablename().equals(tableName) + && (isPartitioned ? scre.getPartitionname().equals("ds=" + partName) : scre.getPartitionname() == null) && + "ready for cleaning".equalsIgnoreCase(scre.getState()) && scre.getType() == CompactionType.ABORT_TXN_CLEANUP && + scre.getErrorMessage().equalsIgnoreCase("Testing retry")); String whereClause = " WHERE \"CQ_DATABASE\" = '" + dbName+ "' AND \"CQ_TABLE\" = '" + tableName + "' AND \"CQ_PARTITION\"" + (isPartitioned ? " = 'ds=" + partName + "'" : " IS NULL") + " AND \"CQ_TYPE\" = 'c' AND \"CQ_STATE\" = 'r'"; - Assert.assertEquals(1, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); String retryRetentionQuery = "SELECT \"CQ_RETRY_RETENTION\" FROM \"COMPACTION_QUEUE\" " + whereClause; Assert.assertEquals(Integer.toString(0), TestTxnDbUtil.queryToString(conf, retryRetentionQuery, false) .replace("\n", "").trim()); - // Cast clob to varchar to get the string output - String retryErrorMsgQuery = "SELECT cast(\"CQ_ERROR_MESSAGE\" as varchar(100)) FROM \"COMPACTION_QUEUE\" " + whereClause; - Assert.assertTrue(TestTxnDbUtil.queryToString(conf, retryErrorMsgQuery, false) - .replace("\n", "").trim().contains("Testing retry")); Mockito.doAnswer(InvocationOnMock::callRealMethod).when(mockedFSRemover).clean(any()); @@ -720,6 +743,6 @@ public void testZeroRetryRetentionTimeForAbortCleanup(boolean isPartitioned) thr cleaner.run(); // The retry entry should be removed since retry conditions are achieved because retry retention time is 0. - Assert.assertEquals(0, TestTxnDbUtil.countQueryAgent(conf, "SELECT COUNT(*) FROM \"COMPACTION_QUEUE\" " + whereClause)); + Assert.assertEquals(0, txnHandler.showCompact(new ShowCompactRequest()).getCompactsSize()); } } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp index 01b19df64ce7..b3d17d959abd 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.cpp @@ -240,13 +240,13 @@ int _kCompactionTypeValues[] = { CompactionType::MINOR, CompactionType::MAJOR, CompactionType::REBALANCE, - CompactionType::ABORT_CLEANUP + CompactionType::ABORT_TXN_CLEANUP }; const char* _kCompactionTypeNames[] = { "MINOR", "MAJOR", "REBALANCE", - "ABORT_CLEANUP" + "ABORT_TXN_CLEANUP" }; const std::map _CompactionType_VALUES_TO_NAMES(::apache::thrift::TEnumIterator(4, _kCompactionTypeValues, _kCompactionTypeNames), ::apache::thrift::TEnumIterator(-1, nullptr, nullptr)); diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h index 93f099bef108..6122d532f689 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-cpp/hive_metastore_types.h @@ -128,7 +128,7 @@ struct CompactionType { MINOR = 1, MAJOR = 2, REBALANCE = 3, - ABORT_CLEANUP = 4 + ABORT_TXN_CLEANUP = 4 }; }; diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java index a7efdff16787..0dabc2868c73 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-javabean/org/apache/hadoop/hive/metastore/api/CompactionType.java @@ -12,7 +12,7 @@ public enum CompactionType implements org.apache.thrift.TEnum { MINOR(1), MAJOR(2), REBALANCE(3), - ABORT_CLEANUP(4); + ABORT_TXN_CLEANUP(4); private final int value; @@ -41,7 +41,7 @@ public static CompactionType findByValue(int value) { case 3: return REBALANCE; case 4: - return ABORT_CLEANUP; + return ABORT_TXN_CLEANUP; default: return null; } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php index 688130711d39..083d81ee0614 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-php/metastore/CompactionType.php @@ -24,13 +24,13 @@ final class CompactionType const REBALANCE = 3; - const ABORT_CLEANUP = 4; + const ABORT_TXN_CLEANUP = 4; static public $__names = array( 1 => 'MINOR', 2 => 'MAJOR', 3 => 'REBALANCE', - 4 => 'ABORT_CLEANUP', + 4 => 'ABORT_TXN_CLEANUP', ); } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py index 53db15c9a5ef..6625b0e283e8 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-py/hive_metastore/ttypes.py @@ -156,20 +156,20 @@ class CompactionType(object): MINOR = 1 MAJOR = 2 REBALANCE = 3 - ABORT_CLEANUP = 4 + ABORT_TXN_CLEANUP = 4 _VALUES_TO_NAMES = { 1: "MINOR", 2: "MAJOR", 3: "REBALANCE", - 4: "ABORT_CLEANUP", + 4: "ABORT_TXN_CLEANUP", } _NAMES_TO_VALUES = { "MINOR": 1, "MAJOR": 2, "REBALANCE": 3, - "ABORT_CLEANUP": 4, + "ABORT_TXN_CLEANUP": 4, } diff --git a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb index 6f014cf0466c..33206ae1f213 100644 --- a/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb +++ b/standalone-metastore/metastore-common/src/gen/thrift/gen-rb/hive_metastore_types.rb @@ -71,9 +71,9 @@ module CompactionType MINOR = 1 MAJOR = 2 REBALANCE = 3 - ABORT_CLEANUP = 4 - VALUE_MAP = {1 => "MINOR", 2 => "MAJOR", 3 => "REBALANCE", 4 => "ABORT_CLEANUP"} - VALID_VALUES = Set.new([MINOR, MAJOR, REBALANCE, ABORT_CLEANUP]).freeze + ABORT_TXN_CLEANUP = 4 + VALUE_MAP = {1 => "MINOR", 2 => "MAJOR", 3 => "REBALANCE", 4 => "ABORT_TXN_CLEANUP"} + VALID_VALUES = Set.new([MINOR, MAJOR, REBALANCE, ABORT_TXN_CLEANUP]).freeze end module GrantRevokeType diff --git a/standalone-metastore/metastore-common/src/main/protobuf/org/apache/hadoop/hive/metastore/hive_metastore.proto b/standalone-metastore/metastore-common/src/main/protobuf/org/apache/hadoop/hive/metastore/hive_metastore.proto index 3f3787aba9c1..34e6a273dd32 100644 --- a/standalone-metastore/metastore-common/src/main/protobuf/org/apache/hadoop/hive/metastore/hive_metastore.proto +++ b/standalone-metastore/metastore-common/src/main/protobuf/org/apache/hadoop/hive/metastore/hive_metastore.proto @@ -2115,6 +2115,8 @@ enum CompactionType { COMPACTION_TYPE_UNSPECIFIED = 0; COMPACTION_TYPE_MINOR = 1; COMPACTION_TYPE_MAJOR = 2; + COMPACTION_TYPE_REBALANCE = 3; + COMPACTION_TYPE_ABORT_TXN_CLEANUP = 4; } enum DataOperationType { diff --git a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift index 0113c24ad9da..c09811ed97bb 100644 --- a/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift +++ b/standalone-metastore/metastore-common/src/main/thrift/hive_metastore.thrift @@ -202,7 +202,7 @@ enum CompactionType { MINOR = 1, MAJOR = 2, REBALANCE = 3, - ABORT_CLEANUP = 4, + ABORT_TXN_CLEANUP = 4, } enum GrantRevokeType { diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java index 206d690b79d9..3c9e5e8ff851 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/HMSHandler.java @@ -8818,7 +8818,7 @@ public OptionalCompactionInfoStruct find_next_compact2(FindNextCompactRequest rq @Override public void mark_cleaned(CompactionInfoStruct cr) throws MetaException { - getTxnHandler().markCleaned(CompactionInfo.compactionStructToInfo(cr), false); + getTxnHandler().markCleaned(CompactionInfo.compactionStructToInfo(cr)); } @Override diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java deleted file mode 100644 index 356ed88a19ee..000000000000 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/AbortTxnRequestInfo.java +++ /dev/null @@ -1,41 +0,0 @@ -/* - * 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.metastore.txn; - -import org.apache.commons.lang3.builder.ToStringBuilder; - -public class AbortTxnRequestInfo extends CompactionInfo { - public boolean hasRetryEntry; - public long minOpenWriteTxnId = 0; - - public AbortTxnRequestInfo() {} - - public String toString() { - return new ToStringBuilder(this) - .append("dbname", dbname) - .append("tableName", tableName) - .append("partName", partName) - .append("runAs", runAs) - .append("minOpenWriteTxnId", minOpenWriteTxnId) - .append("highestWriteId", highestWriteId) - .append("errorMessage", errorMessage) - .append("retryRetention", retryRetention) - .append("hasRetryEntry", hasRetryEntry) - .build(); - } -} diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java index dda975f17b43..f82abcfb2091 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java @@ -351,4 +351,8 @@ public void setWriteIds(boolean hasUncompactedAborts, Set writeIds) { this.hasUncompactedAborts = hasUncompactedAborts; this.writeIds = writeIds; } + + public boolean isAbortedTxnCleanup() { + return type == CompactionType.ABORT_TXN_CLEANUP; + } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index ac69ca33d4fa..3225fee36ab3 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -95,51 +95,53 @@ class CompactionTxnHandler extends TxnHandler { "AND (\"TXN_STATE\" = " + TxnStatus.ABORTED + " OR \"TXN_STATE\" = " + TxnStatus.COMMITTED + ") " + "AND \"TXN_ID\" < ?"; - // Three inner sub-queries which are under union all and gets the max values for corresponding db, table, partition. + // Three inner sub-queries which are under left-join to fetch the required data for aborted txns. private static final String SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY = - "SELECT \"DB\", \"TBL\", \"PART\", MAX(\"MIN_TXN_START_TIME\"), MAX(\"ABORTED_TXN_COUNT\"), " + - " MAX(\"MIN_OPEN_WRITE_TXNID\"), MAX(\"RETRY_RETENTION\"), MAX(\"RETRY_RECORD_CHECK\") FROM ( " + + "SELECT " + + " \"res1\".\"TC_DATABASE\" AS \"DB\", \"res1\".\"TC_TABLE\" AS \"TBL\", \"res1\".\"TC_PARTITION\" AS \"PART\", " + + " \"res1\".\"MIN_TXN_START_TIME\" AS \"MIN_TXN_START_TIME\", \"res1\".\"ABORTED_TXN_COUNT\" AS \"ABORTED_TXN_COUNT\", " + + " \"res2\".\"MIN_OPEN_WRITE_TXNID\" AS \"MIN_OPEN_WRITE_TXNID\", \"res3\".\"RETRY_RETENTION\" AS \"RETRY_RETENTION\", " + + " \"res3\".\"ID\" AS \"RETRY_CQ_ID\" " + + " FROM " + // First sub-query - Gets the aborted txns with min txn start time, number of aborted txns // for corresponding db, table, partition. - " SELECT \"TC_DATABASE\" AS \"DB\", \"TC_TABLE\" AS \"TBL\", \"TC_PARTITION\" AS \"PART\", " + - " MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", COUNT(*) AS \"ABORTED_TXN_COUNT\", " + - " 0 AS \"MIN_OPEN_WRITE_TXNID\", -1 AS \"RETRY_RETENTION\", 0 AS \"RETRY_RECORD_CHECK\" " + - " FROM \"TXNS\", \"TXN_COMPONENTS\" " + + " ( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", MIN(\"TXN_STARTED\") AS \"MIN_TXN_START_TIME\", " + + " COUNT(*) AS \"ABORTED_TXN_COUNT\" FROM \"TXNS\", \"TXN_COMPONENTS\" " + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.ABORTED + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" %s " + - " UNION ALL " + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" %s ) \"res1\" " + + " LEFT JOIN" + // Second sub-query - Gets the min open txn id for corresponding db, table, partition. - " SELECT \"TC_DATABASE\" AS \"DB\", \"TC_TABLE\" AS \"TBL\", \"TC_PARTITION\" AS \"PART\", " + - " 0 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\", " + - " -1 AS \"RETRY_RETENTION\", 0 AS \"RETRY_RECORD_CHECK\" " + + "( SELECT \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\", MIN(\"TC_TXNID\") AS \"MIN_OPEN_WRITE_TXNID\" " + " FROM \"TXNS\", \"TXN_COMPONENTS\" " + " WHERE \"TXN_ID\" = \"TC_TXNID\" AND \"TXN_STATE\" = " + TxnStatus.OPEN + - " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" " + - " UNION ALL " + + " GROUP BY \"TC_DATABASE\", \"TC_TABLE\", \"TC_PARTITION\" ) \"res2\"" + + " ON \"res1\".\"TC_DATABASE\" = \"res2\".\"TC_DATABASE\"" + + " AND \"res1\".\"TC_TABLE\" = \"res2\".\"TC_TABLE\"" + + " AND (\"res1\".\"TC_PARTITION\" = \"res2\".\"TC_PARTITION\" " + + " OR (\"res1\".\"TC_PARTITION\" IS NULL AND \"res2\".\"TC_PARTITION\" IS NULL)) " + + " LEFT JOIN " + // Third sub-query - Gets the retry entries for corresponding db, table, partition. - " SELECT \"CQ_DATABASE\" AS \"DB\", \"CQ_TABLE\" AS \"TBL\", \"CQ_PARTITION\" AS \"PART\", " + - " -1 AS \"MIN_TXN_START_TIME\", -1 AS \"ABORTED_TXN_COUNT\", 0 AS \"MIN_OPEN_WRITE_TXNID\", " + + "( SELECT \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", MAX(\"CQ_ID\") AS \"ID\", " + " MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RETENTION\", " + " MIN(\"CQ_COMMIT_TIME\") - %s + MAX(\"CQ_RETRY_RETENTION\") AS \"RETRY_RECORD_CHECK\" FROM \"COMPACTION_QUEUE\" " + - " WHERE \"CQ_TYPE\" = " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + - " GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\" " + - ") res GROUP BY \"DB\", \"TBL\", \"PART\""; + " WHERE \"CQ_TYPE\" = " + quoteChar(TxnStore.ABORT_TXN_CLEANUP_TYPE) + + " GROUP BY \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\") \"res3\" " + + " ON \"res1\".\"TC_DATABASE\" = \"res3\".\"CQ_DATABASE\" " + + " AND \"res1\".\"TC_TABLE\" = \"res3\".\"CQ_TABLE\" " + + " AND (\"res1\".\"TC_PARTITION\" = \"res3\".\"CQ_PARTITION\" " + + " OR (\"res1\".\"TC_PARTITION\" IS NULL AND \"res3\".\"CQ_PARTITION\" IS NULL))" + + " WHERE \"res3\".\"RETRY_RECORD_CHECK\" <= 0 OR \"res3\".\"RETRY_RECORD_CHECK\" IS NULL"; private static final String DELETE_ABORT_RETRY_ENTRIES_FROM_COMPACTION_QUEUE = "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\" = ? " + "AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) AND \"CQ_TYPE\" = " - + quoteChar(TxnStore.ABORT_CLEANUP_TYPE); + + quoteChar(TxnStore.ABORT_TXN_CLEANUP_TYPE); private static final String INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE = - "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", " + - " \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", \"CQ_TYPE\", \"CQ_ID\", \"CQ_STATE\", \"CQ_COMMIT_TIME\") " + + "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + + "\"CQ_TYPE\", \"CQ_STATE\", \"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", \"CQ_COMMIT_TIME\") " + " VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s)"; - private static final String UPDATE_ABORT_RETRY_ENTRY_IN_COMPACTION_QUEUE = - "UPDATE \"COMPACTION_QUEUE\" SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\" = ? " + - " WHERE \"CQ_DATABASE\" = ? AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) " + - " AND \"CQ_TYPE\" = ?"; - public CompactionTxnHandler() { } @@ -303,9 +305,7 @@ public CompactionInfo findNextToCompact(FindNextCompactRequest rqst) throws Meta StringBuilder sb = new StringBuilder(); sb.append("SELECT \"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + "\"CQ_TYPE\", \"CQ_POOL_NAME\", \"CQ_NUMBER_OF_BUCKETS\", \"CQ_ORDER_BY\", " + - "\"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" " + - "WHERE \"CQ_TYPE\" != " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + - " AND \"CQ_STATE\" = " + quoteChar(INITIATED_STATE) + " AND "); + "\"CQ_TBLPROPERTIES\" FROM \"COMPACTION_QUEUE\" WHERE \"CQ_STATE\" = " + quoteChar(INITIATED_STATE) + " AND "); boolean hasPoolName = StringUtils.isNotBlank(rqst.getPoolName()); if(hasPoolName) { sb.append("\"CQ_POOL_NAME\"=?"); @@ -450,7 +450,7 @@ public List findReadyToClean(long minOpenTxnWaterMark, long rete * the uncompacted deltas. This way the cleaner can clean up everything that was made obsolete by this compaction. */ String whereClause = " WHERE \"CQ_STATE\" = " + quoteChar(READY_FOR_CLEANING) + - " AND \"CQ_TYPE\" != " + quoteChar(TxnStore.ABORT_CLEANUP_TYPE) + + " AND \"CQ_TYPE\" != " + quoteChar(TxnStore.ABORT_TXN_CLEANUP_TYPE) + " AND (\"CQ_COMMIT_TIME\" < (" + getEpochFn(dbProduct) + " - \"CQ_RETRY_RETENTION\" - " + retentionTime + ") OR \"CQ_COMMIT_TIME\" IS NULL)"; String queryStr = @@ -523,16 +523,16 @@ public List findReadyToClean(long minOpenTxnWaterMark, long rete @Override @RetrySemantics.ReadOnly - public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException { + public List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException { try { - List readyToCleanAborts = new ArrayList<>(); + List readyToCleanAborts = new ArrayList<>(); try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction); Statement stmt = dbConn.createStatement()) { boolean checkAbortedTimeThreshold = abortedTimeThreshold >= 0; String sCheckAborted = String.format(SELECT_ABORTS_WITH_MIN_OPEN_WRITETXN_QUERY, checkAbortedTimeThreshold ? "" : " HAVING COUNT(*) > " + abortedThreshold, getEpochFn(dbProduct)); - LOG.info("Going to execute query <{}>", sCheckAborted); + LOG.debug("Going to execute query <{}>", sCheckAborted); try (ResultSet rs = stmt.executeQuery(sCheckAborted)) { long systemTime = System.currentTimeMillis(); while (rs.next()) { @@ -540,22 +540,18 @@ public List findReadyToCleanAborts(long abortedTimeThreshol checkAbortedTimeThreshold && rs.getLong(4) + abortedTimeThreshold < systemTime; int numAbortedTxns = rs.getInt(5); if (numAbortedTxns > abortedThreshold || pastTimeThreshold) { - AbortTxnRequestInfo info = new AbortTxnRequestInfo(); + CompactionInfo info = new CompactionInfo(); info.dbname = rs.getString(1); info.tableName = rs.getString(2); info.partName = rs.getString(3); // In this case, this field contains min open write txn ID. - info.minOpenWriteTxnId = rs.getLong(6) > 0 ? rs.getLong(6) : Long.MAX_VALUE; + info.txnId = rs.getLong(6) > 0 ? rs.getLong(6) : Long.MAX_VALUE; // The specific type, state assigned to abort cleanup. - info.type = CompactionType.ABORT_CLEANUP; + info.type = CompactionType.ABORT_TXN_CLEANUP; info.state = READY_FOR_CLEANING; info.retryRetention = rs.getLong(7); - // Do not add entry when retry record check is > 0. - if (rs.getLong(8) <= 0) { - // In this case, this flag is used for checking whether a retry entry needs to be inserted or updated. - info.hasRetryEntry = info.retryRetention >= 0; - readyToCleanAborts.add(info); - } + info.id = rs.getLong(8); + readyToCleanAborts.add(info); } } } @@ -670,7 +666,7 @@ private void setCleanerStart(Connection dbConn, CompactionInfo info, Long timest */ @Override @RetrySemantics.CannotRetry - public void markCleaned(CompactionInfo info, boolean isAbortOnly) throws MetaException { + public void markCleaned(CompactionInfo info) throws MetaException { LOG.debug("Running markCleaned with CompactionInfo: {}", info); try { Connection dbConn = null; @@ -678,7 +674,7 @@ public void markCleaned(CompactionInfo info, boolean isAbortOnly) throws MetaExc ResultSet rs = null; try { dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction); - if (!isAbortOnly) { + if (!info.isAbortedTxnCleanup()) { String s = "INSERT INTO \"COMPLETED_COMPACTIONS\"(\"CC_ID\", \"CC_DATABASE\", " + "\"CC_TABLE\", \"CC_PARTITION\", \"CC_STATE\", \"CC_TYPE\", \"CC_TBLPROPERTIES\", \"CC_WORKER_ID\", " + "\"CC_START\", \"CC_END\", \"CC_RUN_AS\", \"CC_HIGHEST_WRITE_ID\", \"CC_META_INFO\", " @@ -745,13 +741,13 @@ public void markCleaned(CompactionInfo info, boolean isAbortOnly) throws MetaExc LOG.error("Unable to delete from compaction queue " + e.getMessage()); LOG.debug("Going to rollback"); rollbackDBConn(dbConn); - checkRetryable(e, "markCleaned(" + info + "," + isAbortOnly + ")"); + checkRetryable(e, "markCleaned(" + info + ")"); throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } finally { close(rs, pStmt, dbConn); } } catch (RetryException e) { - markCleaned(info, isAbortOnly); + markCleaned(info); } } @@ -762,7 +758,8 @@ private void removeTxnComponents(Connection dbConn, CompactionInfo info) throws /* * Remove all abort retry associated metadata of table/partition in the COMPACTION_QUEUE both when compaction * or abort cleanup is successful. We don't want a situation wherein we have a abort retry entry for a table - * but no corresponding entry in TXN_COMPONENTS table. + * but no corresponding entry in TXN_COMPONENTS table. Successful compaction will delete + * the retry metadata, so that abort cleanup is retried again (an optimistic retry approach). */ removeAbortRetryEntries(dbConn, info); @@ -993,7 +990,7 @@ public void removeDuplicateCompletedTxnComponents() throws MetaException { * The committed txns are left there for TXN_OPENTXN_TIMEOUT window period intentionally. * The reason such aborted txns exist can be that now work was done in this txn * (e.g. Streaming opened TransactionBatch and abandoned it w/o doing any work) - * or due to {@link #markCleaned(CompactionInfo, boolean)} being called. + * or due to {@link #markCleaned(CompactionInfo)} being called. */ @Override @RetrySemantics.SafeToRetry @@ -1321,47 +1318,51 @@ private void removeAbortRetryEntries(Connection dbConn, CompactionInfo info) thr } } - private void setAbortRetryRetentionTimeOnError(Connection dbConn, AbortTxnRequestInfo info) throws MetaException, RetryException { - String query = info.hasRetryEntry ? UPDATE_ABORT_RETRY_ENTRY_IN_COMPACTION_QUEUE : - String.format(INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE, getEpochFn(dbProduct)); - try (PreparedStatement pStmt = dbConn.prepareStatement(query); Statement stmt = dbConn.createStatement()) { - pStmt.setLong(1, info.retryRetention); - pStmt.setString(2, info.errorMessage); - pStmt.setString(3, info.dbname); - pStmt.setString(4, info.tableName); + private void insertAbortRetryRetentionTimeOnError(Connection dbConn, CompactionInfo info) throws MetaException, SQLException { + String query = String.format(INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE, getEpochFn(dbProduct)); + TxnStore.MutexAPI.LockHandle handle = null; + try (PreparedStatement pStmt = dbConn.prepareStatement(query); + Statement stmt = dbConn.createStatement()) { + lockInternal(); + /** + * MUTEX_KEY.CompactionScheduler lock ensures that there is only 1 entry in + * Initiated/Working state for any resource. This ensures that we don't run concurrent + * compactions for any resource. + */ + handle = getMutexAPI().acquireLock(MUTEX_KEY.CompactionScheduler.name()); + long id = generateCompactionQueueId(stmt); + pStmt.setLong(1, id); + pStmt.setString(2, info.dbname); + pStmt.setString(3, info.tableName); if (info.partName != null) { - pStmt.setString(5, info.partName); + pStmt.setString(4, info.partName); } else { // Since the type of 'CQ_PARTITION' column is varchar. // Hence, setting null for VARCHAR type. - pStmt.setNull(5, Types.VARCHAR); - } - pStmt.setString(6, Character.toString(thriftCompactionType2DbType(info.type))); - if (!info.hasRetryEntry) { - long id = generateCompactionQueueId(stmt); - pStmt.setLong(7, id); - pStmt.setString(8, Character.toString(info.state)); + pStmt.setNull(4, Types.VARCHAR); } + pStmt.setString(5, Character.toString(thriftCompactionType2DbType(info.type))); + pStmt.setString(6, Character.toString(info.state)); + pStmt.setLong(7, info.retryRetention); + pStmt.setString(8, info.errorMessage); int updCnt = pStmt.executeUpdate(); if (updCnt == 0) { LOG.error("Unable to update/insert compaction queue record: {}. updCnt={}", info, updCnt); dbConn.rollback(); - if (info.hasRetryEntry) { - throw new MetaException("No abort retry record with CQ_DATABASE=" + info.dbname + ", CQ_TABLE=" - + info.tableName + ", CQ_PARTITION" + info.partName + " found in COMPACTION_QUEUE"); - } else { - throw new MetaException("Unable to insert abort retry entry into COMPACTION QUEUE: " + - " CQ_DATABASE=" + info.dbname + ", CQ_TABLE=" + info.tableName + ", CQ_PARTITION" + info.partName); - } + throw new MetaException("Unable to insert abort retry entry into COMPACTION QUEUE: " + + " CQ_DATABASE=" + info.dbname + ", CQ_TABLE=" + info.tableName + ", CQ_PARTITION" + info.partName); } LOG.debug("Going to commit"); dbConn.commit(); } catch (SQLException e) { - LOG.error("Unable to update compaction queue: " + e.getMessage()); + LOG.error("Unable to update compaction queue: {}", e.getMessage()); rollbackDBConn(dbConn); - checkRetryable(e, "setAbortRetryRetentionTimeOnError(" + info + ")"); - throw new MetaException("Unable to update compaction queue: " + - e.getMessage()); + throw e; + } finally { + if (handle != null) { + handle.releaseLocks(); + } + unlockInternal(); } } @@ -1652,52 +1653,40 @@ public void markRefused(CompactionInfo info) throws MetaException { @Override @RetrySemantics.CannotRetry - public void setAbortCleanerRetryRetentionTimeOnError(AbortTxnRequestInfo info) throws MetaException { + public void insertOrSetCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { try { try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { - setAbortRetryRetentionTimeOnError(dbConn, info); - } catch (SQLException e) { - LOG.error(DB_FAILED_TO_CONNECT + e.getMessage()); - checkRetryable(e, "setAbortCleanerRetryRetentionTimeOnError(" + info + ")"); - throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); - } - } catch (RetryException e) { - setAbortCleanerRetryRetentionTimeOnError(info); - } - } - - @Override - @RetrySemantics.CannotRetry - public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { - try { - try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { - try (PreparedStatement stmt = dbConn.prepareStatement("UPDATE \"COMPACTION_QUEUE\" " + - "SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?")) { - stmt.setLong(1, info.retryRetention); - stmt.setString(2, info.errorMessage); - stmt.setLong(3, info.id); - int updCnt = stmt.executeUpdate(); - if (updCnt != 1) { - LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); - dbConn.rollback(); - throw new MetaException("No record with CQ_ID=" + info.id + " found in COMPACTION_QUEUE"); + if (info.isAbortedTxnCleanup() && info.id == 0) { + insertAbortRetryRetentionTimeOnError(dbConn, info); + } else { + try (PreparedStatement stmt = dbConn.prepareStatement("UPDATE \"COMPACTION_QUEUE\" " + + "SET \"CQ_RETRY_RETENTION\" = ?, \"CQ_ERROR_MESSAGE\"= ? WHERE \"CQ_ID\" = ?")) { + stmt.setLong(1, info.retryRetention); + stmt.setString(2, info.errorMessage); + stmt.setLong(3, info.id); + int updCnt = stmt.executeUpdate(); + if (updCnt != 1) { + LOG.error("Unable to update compaction queue record: {}. updCnt={}", info, updCnt); + dbConn.rollback(); + throw new MetaException("No record with CQ_ID=" + info.id + " found in COMPACTION_QUEUE"); + } + LOG.debug("Going to commit"); + dbConn.commit(); + } catch (SQLException e) { + LOG.error("Unable to update compaction queue: " + e.getMessage()); + rollbackDBConn(dbConn); + checkRetryable(e, "insertOrSetCleanerRetryRetentionTimeOnError(" + info + ")"); + throw new MetaException("Unable to update compaction queue: " + + e.getMessage()); } - LOG.debug("Going to commit"); - dbConn.commit(); - } catch (SQLException e) { - LOG.error("Unable to update compaction queue: " + e.getMessage()); - rollbackDBConn(dbConn); - checkRetryable(e, "setCleanerRetryRetentionTimeOnError(" + info + ")"); - throw new MetaException("Unable to update compaction queue: " + - e.getMessage()); } } catch (SQLException e) { LOG.error(DB_FAILED_TO_CONNECT + e.getMessage()); - checkRetryable(e, "setCleanerRetryRetentionTimeOnError(" + info + ")"); + checkRetryable(e, "insertOrSetCleanerRetryRetentionTimeOnError(" + info + ")"); throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - setCleanerRetryRetentionTimeOnError(info); + insertOrSetCleanerRetryRetentionTimeOnError(info); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java index fa194c11b314..77be31dbced8 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnHandler.java @@ -6084,12 +6084,12 @@ static String quoteChar(char c) { * Select ... For Update to sequence operations properly. In practice that means when running * with Derby database. See more notes at class level. */ - private void lockInternal() { + protected void lockInternal() { if(dbProduct.isDERBY()) { derbyLock.lock(); } } - private void unlockInternal() { + protected void unlockInternal() { if(dbProduct.isDERBY()) { derbyLock.unlock(); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index 5c919b4df10e..e14fb9e94975 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -124,7 +124,7 @@ enum MUTEX_KEY { char MAJOR_TYPE = 'a'; char MINOR_TYPE = 'i'; char REBALANCE_TYPE = 'r'; - char ABORT_CLEANUP_TYPE = 'c'; + char ABORT_TXN_CLEANUP_TYPE = 'c'; String[] COMPACTION_STATES = new String[] {INITIATED_RESPONSE, WORKING_RESPONSE, CLEANING_RESPONSE, FAILED_RESPONSE, SUCCEEDED_RESPONSE, DID_NOT_INITIATE_RESPONSE, REFUSED_RESPONSE }; @@ -527,7 +527,7 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * @throws MetaException */ @RetrySemantics.ReadOnly - List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException; + List findReadyToCleanAborts(long abortedTimeThreshold, int abortedThreshold) throws MetaException; /** * Sets the cleaning start time for a particular compaction @@ -550,10 +550,9 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * it has been compacted. * * @param info info on the compaction entry to remove - * @param isAbortOnly whether to cleanup only abort related cleanup information */ @RetrySemantics.CannotRetry - void markCleaned(CompactionInfo info, boolean isAbortOnly) throws MetaException; + void markCleaned(CompactionInfo info) throws MetaException; /** * Mark a compaction entry as failed. This will move it to the compaction history queue with a @@ -575,21 +574,12 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT /** * Stores the value of {@link CompactionInfo#retryRetention} and {@link CompactionInfo#errorMessage} fields - * of the CompactionInfo in the HMS database. + * of the CompactionInfo either by inserting or updating the fields in the HMS database. * @param info The {@link CompactionInfo} object holding the values. * @throws MetaException */ @RetrySemantics.CannotRetry - void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; - - /** - * Stores the value of {@link AbortTxnRequestInfo#retryRetention} and {@link AbortTxnRequestInfo#errorMessage} fields - * of the AbortTxnRequestInfo in the HMS database (specifically in TXN_CLEANUP_QUEUE table). - * @param info - * @throws MetaException - */ - @RetrySemantics.CannotRetry - void setAbortCleanerRetryRetentionTimeOnError(AbortTxnRequestInfo info) throws MetaException; + void insertOrSetCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by @@ -607,7 +597,7 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT /** * Clean up aborted or committed transactions from txns that have no components in txn_components. The reason such * txns exist can be that no work was done in this txn (e.g. Streaming opened TransactionBatch and - * abandoned it w/o doing any work) or due to {@link #markCleaned(CompactionInfo, boolean)} being called, + * abandoned it w/o doing any work) or due to {@link #markCleaned(CompactionInfo)} being called, * or the delete from the txns was delayed because of TXN_OPENTXN_TIMEOUT window. */ @RetrySemantics.SafeToRetry diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java index 854d7377318e..1eedef276a36 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnUtils.java @@ -586,8 +586,8 @@ public static CompactionType dbCompactionType2ThriftType(char dbValue) throws Me return CompactionType.MINOR; case TxnStore.REBALANCE_TYPE: return CompactionType.REBALANCE; - case TxnStore.ABORT_CLEANUP_TYPE: - return CompactionType.ABORT_CLEANUP; + case TxnStore.ABORT_TXN_CLEANUP_TYPE: + return CompactionType.ABORT_TXN_CLEANUP; default: throw new MetaException("Unexpected compaction type " + dbValue); } @@ -601,8 +601,8 @@ public static Character thriftCompactionType2DbType(CompactionType ct) throws Me return TxnStore.MINOR_TYPE; case REBALANCE: return TxnStore.REBALANCE_TYPE; - case ABORT_CLEANUP: - return TxnStore.ABORT_CLEANUP_TYPE; + case ABORT_TXN_CLEANUP: + return TxnStore.ABORT_TXN_CLEANUP_TYPE; default: throw new MetaException("Unexpected compaction type " + ct); } From 247595bca2733d385469c402e0b46736e66502b4 Mon Sep 17 00:00:00 2001 From: Sourabh Badhya Date: Thu, 8 Jun 2023 14:00:04 +0530 Subject: [PATCH 6/6] Address review comments - 5 --- .../compactor/handler/AbortedTxnCleaner.java | 15 +++++++++------ .../ql/txn/compactor/handler/TaskHandler.java | 4 ++-- .../handler/TestAbortedTxnCleaner.java | 2 +- .../hive/metastore/txn/CompactionInfo.java | 2 ++ .../metastore/txn/CompactionTxnHandler.java | 18 +++++++++--------- .../hadoop/hive/metastore/txn/TxnStore.java | 2 +- 6 files changed, 24 insertions(+), 19 deletions(-) diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java index 65e7cf71b9cf..acd4519d0fe3 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/AbortedTxnCleaner.java @@ -58,15 +58,18 @@ public AbortedTxnCleaner(HiveConf conf, TxnStore txnHandler, /** The following cleanup is based on the following idea -
- 1. Aborted cleanup is independent of compaction. This is because directories which are written by - aborted txns are not visible by any open txns. It is only visible while determining the AcidState (which - only sees the aborted deltas and does not read the file).

+ Aborted cleanup is independent of compaction. This is because directories which are written by + aborted txns are not visible by any open txns in most cases. The one case, wherein abort deltas + are visible for open txns are streaming aborts wherein, a single delta file can have writes + from committed and aborted txns. These deltas are also referred to as uncompacted aborts. + In such cases, we do not delete uncompacted aborts or its associated metadata. The following algorithm is used to clean the set of aborted directories -
a. Find the list of entries which are suitable for cleanup (This is done in {@link TxnStore#findReadyToCleanAborts(long, int)}).
b. If the table/partition does not exist, then remove the associated aborted entry in TXN_COMPONENTS table.
- c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID
- d. Fetch the aborted directories and delete the directories.
+ c. Get the AcidState of the table by using the min open txnID, database name, tableName, partition name, highest write ID. + The construction of AcidState helps in identifying obsolete/aborted directories in the table/partition.
+ d. Fetch the obsolete/aborted directories and delete the directories using the AcidState.
e. Fetch the aborted write IDs from the AcidState and use it to delete the associated metadata in the TXN_COMPONENTS table. **/ @Override @@ -80,7 +83,7 @@ public List getTasks() throws MetaException { if (!readyToCleanAborts.isEmpty()) { return readyToCleanAborts.stream().map(info -> ThrowingRunnable.unchecked(() -> - clean(info, info.txnId, metricsEnabled))) + clean(info, info.minOpenWriteTxnId, metricsEnabled))) .collect(Collectors.toList()); } return Collections.emptyList(); diff --git a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java index aa8b738834ad..ff35ce484237 100644 --- a/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java +++ b/ql/src/java/org/apache/hadoop/hive/ql/txn/compactor/handler/TaskHandler.java @@ -175,7 +175,7 @@ protected void handleCleanerAttemptFailure(CompactionInfo info, String errorMess if (info.isAbortedTxnCleanup()) { info.retryRetention = info.retryRetention > 0 ? info.retryRetention * 2 : defaultRetention; info.errorMessage = errorMessage; - txnHandler.insertOrSetCleanerRetryRetentionTimeOnError(info); + txnHandler.setCleanerRetryRetentionTimeOnError(info); } else { if (info.retryRetention > 0) { cleanAttempts = (int) (Math.log(info.retryRetention / defaultRetention) / Math.log(2)) + 1; @@ -186,7 +186,7 @@ protected void handleCleanerAttemptFailure(CompactionInfo info, String errorMess } else { //Calculate retry retention time and update record. info.retryRetention = (long) Math.pow(2, cleanAttempts) * defaultRetention; - txnHandler.insertOrSetCleanerRetryRetentionTimeOnError(info); + txnHandler.setCleanerRetryRetentionTimeOnError(info); } } } diff --git a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java index 8da8c5b15835..e0a2a1f5ed32 100644 --- a/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java +++ b/ql/src/test/org/apache/hadoop/hive/ql/txn/compactor/handler/TestAbortedTxnCleaner.java @@ -412,7 +412,7 @@ public void testRetryEntryOnFailures(boolean isPartitioned) throws Exception { cleaner.setCleanupHandlers(Arrays.asList(mockedTaskHandler)); cleaner.run(); - Mockito.verify(mockedTxnHandler, Mockito.times(1)).insertOrSetCleanerRetryRetentionTimeOnError(any(CompactionInfo.class)); + Mockito.verify(mockedTxnHandler, Mockito.times(1)).setCleanerRetryRetentionTimeOnError(any(CompactionInfo.class)); ShowCompactResponse scr = txnHandler.showCompact(new ShowCompactRequest()); Assert.assertEquals(1, scr.getCompactsSize()); ShowCompactResponseElement scre = scr.getCompacts().get(0); diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java index f82abcfb2091..b238a98f44aa 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionInfo.java @@ -67,6 +67,7 @@ public class CompactionInfo implements Comparable { public String poolName; public int numberOfBuckets = 0; public String orderByClause; + public long minOpenWriteTxnId = 0; /** * The highest write id that the compaction job will pay attention to. @@ -178,6 +179,7 @@ public String toString() { .append("poolName", poolName) .append("numberOfBuckets", numberOfBuckets) .append("orderByClause", orderByClause) + .append("minOpenWriteTxnId", minOpenWriteTxnId) .build(); } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java index 3225fee36ab3..4955f62d58b3 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/CompactionTxnHandler.java @@ -133,14 +133,14 @@ class CompactionTxnHandler extends TxnHandler { " WHERE \"res3\".\"RETRY_RECORD_CHECK\" <= 0 OR \"res3\".\"RETRY_RECORD_CHECK\" IS NULL"; private static final String DELETE_ABORT_RETRY_ENTRIES_FROM_COMPACTION_QUEUE = - "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\" = ? " + - "AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) AND \"CQ_TYPE\" = " - + quoteChar(TxnStore.ABORT_TXN_CLEANUP_TYPE); + "DELETE FROM \"COMPACTION_QUEUE\" WHERE \"CQ_DATABASE\" = ? " + + " AND \"CQ_TABLE\" = ? AND (\"CQ_PARTITION\" = ? OR \"CQ_PARTITION\" IS NULL) AND \"CQ_TYPE\" = " + + quoteChar(TxnStore.ABORT_TXN_CLEANUP_TYPE); private static final String INSERT_ABORT_RETRY_ENTRY_INTO_COMPACTION_QUEUE = - "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + - "\"CQ_TYPE\", \"CQ_STATE\", \"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", \"CQ_COMMIT_TIME\") " + - " VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s)"; + "INSERT INTO \"COMPACTION_QUEUE\" (\"CQ_ID\", \"CQ_DATABASE\", \"CQ_TABLE\", \"CQ_PARTITION\", " + + " \"CQ_TYPE\", \"CQ_STATE\", \"CQ_RETRY_RETENTION\", \"CQ_ERROR_MESSAGE\", \"CQ_COMMIT_TIME\") " + + " VALUES (?, ?, ?, ?, ?, ?, ?, ?, %s)"; public CompactionTxnHandler() { } @@ -545,7 +545,7 @@ public List findReadyToCleanAborts(long abortedTimeThreshold, in info.tableName = rs.getString(2); info.partName = rs.getString(3); // In this case, this field contains min open write txn ID. - info.txnId = rs.getLong(6) > 0 ? rs.getLong(6) : Long.MAX_VALUE; + info.minOpenWriteTxnId = rs.getLong(6) > 0 ? rs.getLong(6) : Long.MAX_VALUE; // The specific type, state assigned to abort cleanup. info.type = CompactionType.ABORT_TXN_CLEANUP; info.state = READY_FOR_CLEANING; @@ -1653,7 +1653,7 @@ public void markRefused(CompactionInfo info) throws MetaException { @Override @RetrySemantics.CannotRetry - public void insertOrSetCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { + public void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException { try { try (Connection dbConn = getDbConn(Connection.TRANSACTION_READ_COMMITTED, connPoolCompaction)) { if (info.isAbortedTxnCleanup() && info.id == 0) { @@ -1686,7 +1686,7 @@ public void insertOrSetCleanerRetryRetentionTimeOnError(CompactionInfo info) thr throw new MetaException(DB_FAILED_TO_CONNECT + e.getMessage()); } } catch (RetryException e) { - insertOrSetCleanerRetryRetentionTimeOnError(info); + setCleanerRetryRetentionTimeOnError(info); } } diff --git a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java index e14fb9e94975..7be1b0cc52b1 100644 --- a/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java +++ b/standalone-metastore/metastore-server/src/main/java/org/apache/hadoop/hive/metastore/txn/TxnStore.java @@ -579,7 +579,7 @@ Set findPotentialCompactions(int abortedThreshold, long abortedT * @throws MetaException */ @RetrySemantics.CannotRetry - void insertOrSetCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; + void setCleanerRetryRetentionTimeOnError(CompactionInfo info) throws MetaException; /** * Clean up entries from TXN_TO_WRITE_ID table less than min_uncommited_txnid as found by